diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index ae5e9789dd941..2b6867b0ff967 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -238,6 +238,16 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] transformAllExpressionsWithPruning(AlwaysProcess.fn, UnknownRuleId)(rule) } + /** + * A variant of [[transformAllExpressions]] which considers plan nodes inside subqueries as well. + */ + def transformAllExpressionsWithSubqueries( + rule: PartialFunction[Expression, Expression]): this.type = { + transformWithSubqueries { + case q => q.transformExpressions(rule).asInstanceOf[PlanType] + }.asInstanceOf[this.type] + } + /** * Returns the result of running [[transformExpressionsWithPruning]] on this node * and all its children. Note that this method skips expressions inside subqueries. 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/ansi/array.sql.out index a9c18ab869025..b71d0aad152fc 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/ansi/array.sql.out @@ -129,13 +129,7 @@ select array_contains(timestamp_array, timestamp '2016-11-15 20:54:00.000'), array_contains(timestamp_array, timestamp '2016-01-01 20:54:00.000') from primitive_arrays -- !query analysis -Project [array_contains(boolean_array#x, true) AS array_contains(boolean_array, true)#x, array_contains(boolean_array#x, false) AS array_contains(boolean_array, false)#x, array_contains(tinyint_array#x, 2) AS array_contains(tinyint_array, 2)#x, array_contains(tinyint_array#x, 0) AS array_contains(tinyint_array, 0)#x, array_contains(smallint_array#x, 2) AS array_contains(smallint_array, 2)#x, array_contains(smallint_array#x, 0) AS array_contains(smallint_array, 0)#x, array_contains(int_array#x, 2) AS array_contains(int_array, 2)#x, array_contains(int_array#x, 0) AS array_contains(int_array, 0)#x, array_contains(bigint_array#x, 2) AS array_contains(bigint_array, 2)#x, array_contains(bigint_array#x, 0) AS array_contains(bigint_array, 0)#x, array_contains(decimal_array#x, 9223372036854775809) AS array_contains(decimal_array, 9223372036854775809)#x, array_contains(decimal_array#x, cast(1 as decimal(19,0))) AS array_contains(decimal_array, 1)#x, array_contains(double_array#x, 2.0) AS array_contains(double_array, 2.0)#x, array_contains(double_array#x, 0.0) AS array_contains(double_array, 0.0)#x, array_contains(float_array#x, cast(2.0 as float)) AS array_contains(float_array, 2.0)#x, array_contains(float_array#x, cast(0.0 as float)) AS array_contains(float_array, 0.0)#x, array_contains(date_array#x, 2016-03-14) AS array_contains(date_array, DATE '2016-03-14')#x, array_contains(date_array#x, 2016-01-01) AS array_contains(date_array, DATE '2016-01-01')#x, array_contains(timestamp_array#x, 2016-11-15 20:54:00) AS array_contains(timestamp_array, TIMESTAMP '2016-11-15 20:54:00')#x, array_contains(timestamp_array#x, 2016-01-01 20:54:00) AS array_contains(timestamp_array, TIMESTAMP '2016-01-01 20:54:00')#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] - +- Project [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] - +- SubqueryAlias primitive_arrays - +- LocalRelation [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] +[Analyzer test output redacted due to nondeterminism] -- !query @@ -235,7 +229,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] @@ -247,70 +241,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 @@ -383,21 +377,21 @@ org.apache.spark.sql.AnalysisException -- !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/ansi/cast.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/cast.sql.out new file mode 100644 index 0000000000000..730704b3cbf63 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/cast.sql.out @@ -0,0 +1,881 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT CAST('1.23' AS int) +-- !query analysis +Project [cast(1.23 as int) AS CAST(1.23 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('1.23' AS long) +-- !query analysis +Project [cast(1.23 as bigint) AS CAST(1.23 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('-4.56' AS int) +-- !query analysis +Project [cast(-4.56 as int) AS CAST(-4.56 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('-4.56' AS long) +-- !query analysis +Project [cast(-4.56 as bigint) AS CAST(-4.56 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('abc' AS int) +-- !query analysis +Project [cast(abc as int) AS CAST(abc AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('abc' AS long) +-- !query analysis +Project [cast(abc as bigint) AS CAST(abc AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('abc' AS float) +-- !query analysis +Project [cast(abc as float) AS CAST(abc AS FLOAT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('abc' AS double) +-- !query analysis +Project [cast(abc as double) AS CAST(abc AS DOUBLE)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('1234567890123' AS int) +-- !query analysis +Project [cast(1234567890123 as int) AS CAST(1234567890123 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('12345678901234567890123' AS long) +-- !query analysis +Project [cast(12345678901234567890123 as bigint) AS CAST(12345678901234567890123 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('' AS int) +-- !query analysis +Project [cast( as int) AS CAST( AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('' AS long) +-- !query analysis +Project [cast( as bigint) AS CAST( AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('' AS float) +-- !query analysis +Project [cast( as float) AS CAST( AS FLOAT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('' AS double) +-- !query analysis +Project [cast( as double) AS CAST( AS DOUBLE)#x] ++- OneRowRelation + + +-- !query +SELECT CAST(NULL AS int) +-- !query analysis +Project [cast(null as int) AS CAST(NULL AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST(NULL AS long) +-- !query analysis +Project [cast(null as bigint) AS CAST(NULL AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('123.a' AS int) +-- !query analysis +Project [cast(123.a as int) AS CAST(123.a AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('123.a' AS long) +-- !query analysis +Project [cast(123.a as bigint) AS CAST(123.a AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('123.a' AS float) +-- !query analysis +Project [cast(123.a as float) AS CAST(123.a AS FLOAT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('123.a' AS double) +-- !query analysis +Project [cast(123.a as double) AS CAST(123.a AS DOUBLE)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('-2147483648' AS int) +-- !query analysis +Project [cast(-2147483648 as int) AS CAST(-2147483648 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('-2147483649' AS int) +-- !query analysis +Project [cast(-2147483649 as int) AS CAST(-2147483649 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('2147483647' AS int) +-- !query analysis +Project [cast(2147483647 as int) AS CAST(2147483647 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('2147483648' AS int) +-- !query analysis +Project [cast(2147483648 as int) AS CAST(2147483648 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('-9223372036854775808' AS long) +-- !query analysis +Project [cast(-9223372036854775808 as bigint) AS CAST(-9223372036854775808 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('-9223372036854775809' AS long) +-- !query analysis +Project [cast(-9223372036854775809 as bigint) AS CAST(-9223372036854775809 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('9223372036854775807' AS long) +-- !query analysis +Project [cast(9223372036854775807 as bigint) AS CAST(9223372036854775807 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('9223372036854775808' AS long) +-- !query analysis +Project [cast(9223372036854775808 as bigint) AS CAST(9223372036854775808 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT HEX(CAST('abc' AS binary)) +-- !query analysis +Project [hex(cast(abc as binary)) AS hex(CAST(abc AS BINARY))#x] ++- OneRowRelation + + +-- !query +SELECT HEX(CAST(CAST(123 AS byte) AS binary)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 analysis +DescribeFunctionCommand org.apache.spark.sql.catalyst.expressions.ExpressionInfo@xxxxxxxx, false + + +-- !query +DESC FUNCTION EXTENDED boolean +-- !query analysis +DescribeFunctionCommand org.apache.spark.sql.catalyst.expressions.ExpressionInfo@xxxxxxxx, true + + +-- !query +SELECT CAST('interval 3 month 1 hour' AS interval) +-- !query analysis +Project [cast(interval 3 month 1 hour as interval) AS CAST(interval 3 month 1 hour AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +SELECT CAST("interval '3-1' year to month" AS interval year to month) +-- !query analysis +Project [cast(interval '3-1' year to month as interval year to month) AS CAST(interval '3-1' year to month AS INTERVAL YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT CAST("interval '3 00:00:01' day to second" AS interval day to second) +-- !query analysis +Project [cast(interval '3 00:00:01' day to second as interval day to second) AS CAST(interval '3 00:00:01' day to second AS INTERVAL DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +SELECT CAST(interval 3 month 1 hour AS string) +-- !query analysis +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 analysis +Project [cast(INTERVAL '3-1' YEAR TO MONTH as string) AS CAST(INTERVAL '3-1' YEAR TO MONTH AS STRING)#x] ++- OneRowRelation + + +-- !query +SELECT CAST(interval 3 day 1 second AS string) +-- !query analysis +Project [cast(INTERVAL '3 00:00:01' DAY TO SECOND as string) AS CAST(INTERVAL '3 00:00:01' DAY TO SECOND AS STRING)#x] ++- OneRowRelation + + +-- !query +select cast(' 1' as tinyint) +-- !query analysis +Project [cast( 1 as tinyint) AS CAST( 1 AS TINYINT)#x] ++- OneRowRelation + + +-- !query +select cast(' 1\t' as tinyint) +-- !query analysis +Project [cast( 1 as tinyint) AS CAST( 1 AS TINYINT)#x] ++- OneRowRelation + + +-- !query +select cast(' 1' as smallint) +-- !query analysis +Project [cast( 1 as smallint) AS CAST( 1 AS SMALLINT)#x] ++- OneRowRelation + + +-- !query +select cast(' 1' as INT) +-- !query analysis +Project [cast( 1 as int) AS CAST( 1 AS INT)#x] ++- OneRowRelation + + +-- !query +select cast(' 1' as bigint) +-- !query analysis +Project [cast( 1 as bigint) AS CAST( 1 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +select cast(' 1' as float) +-- !query analysis +Project [cast( 1 as float) AS CAST( 1 AS FLOAT)#x] ++- OneRowRelation + + +-- !query +select cast(' 1 ' as DOUBLE) +-- !query analysis +Project [cast( 1 as double) AS CAST( 1 AS DOUBLE)#x] ++- OneRowRelation + + +-- !query +select cast('1.0 ' as DEC) +-- !query analysis +Project [cast(1.0 as decimal(10,0)) AS CAST(1.0 AS DECIMAL(10,0))#x] ++- OneRowRelation + + +-- !query +select cast('1中文' as tinyint) +-- !query analysis +Project [cast(1中文 as tinyint) AS CAST(1中文 AS TINYINT)#x] ++- OneRowRelation + + +-- !query +select cast('1中文' as smallint) +-- !query analysis +Project [cast(1中文 as smallint) AS CAST(1中文 AS SMALLINT)#x] ++- OneRowRelation + + +-- !query +select cast('1中文' as INT) +-- !query analysis +Project [cast(1中文 as int) AS CAST(1中文 AS INT)#x] ++- OneRowRelation + + +-- !query +select cast('中文1' as bigint) +-- !query analysis +Project [cast(中文1 as bigint) AS CAST(中文1 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +select cast('1中文' as bigint) +-- !query analysis +Project [cast(1中文 as bigint) AS CAST(1中文 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +select cast('\t\t true \n\r ' as boolean) +-- !query analysis +Project [cast( true + as boolean) AS CAST( true + AS BOOLEAN)#x] ++- OneRowRelation + + +-- !query +select cast('\t\n false \t\r' as boolean) +-- !query analysis +Project [cast( + false as boolean) AS CAST( + false AS BOOLEAN)#x] ++- OneRowRelation + + +-- !query +select cast('\t\n xyz \t\r' as boolean) +-- !query analysis +Project [cast( + xyz as boolean) AS CAST( + xyz AS BOOLEAN)#x] ++- OneRowRelation + + +-- !query +select cast('23.45' as decimal(4, 2)) +-- !query analysis +Project [cast(23.45 as decimal(4,2)) AS CAST(23.45 AS DECIMAL(4,2))#x] ++- OneRowRelation + + +-- !query +select cast('123.45' as decimal(4, 2)) +-- !query analysis +Project [cast(123.45 as decimal(4,2)) AS CAST(123.45 AS DECIMAL(4,2))#x] ++- OneRowRelation + + +-- !query +select cast('xyz' as decimal(4, 2)) +-- !query analysis +Project [cast(xyz as decimal(4,2)) AS CAST(xyz AS DECIMAL(4,2))#x] ++- OneRowRelation + + +-- !query +select cast('2022-01-01' as date) +-- !query analysis +Project [cast(2022-01-01 as date) AS CAST(2022-01-01 AS DATE)#x] ++- OneRowRelation + + +-- !query +select cast('a' as date) +-- !query analysis +Project [cast(a as date) AS CAST(a AS DATE)#x] ++- OneRowRelation + + +-- !query +select cast('2022-01-01 00:00:00' as timestamp) +-- !query analysis +Project [cast(2022-01-01 00:00:00 as timestamp) AS CAST(2022-01-01 00:00:00 AS TIMESTAMP)#x] ++- OneRowRelation + + +-- !query +select cast('a' as timestamp) +-- !query analysis +Project [cast(a as timestamp) AS CAST(a AS TIMESTAMP)#x] ++- OneRowRelation + + +-- !query +select cast('2022-01-01 00:00:00' as timestamp_ntz) +-- !query analysis +Project [cast(2022-01-01 00:00:00 as timestamp_ntz) AS CAST(2022-01-01 00:00:00 AS TIMESTAMP_NTZ)#x] ++- OneRowRelation + + +-- !query +select cast('a' as timestamp_ntz) +-- !query analysis +Project [cast(a as timestamp_ntz) AS CAST(a AS TIMESTAMP_NTZ)#x] ++- OneRowRelation + + +-- !query +select cast(cast('inf' as double) as timestamp) +-- !query analysis +Project [cast(cast(inf as double) as timestamp) AS CAST(CAST(inf AS DOUBLE) AS TIMESTAMP)#x] ++- OneRowRelation + + +-- !query +select cast(cast('inf' as float) as timestamp) +-- !query analysis +Project [cast(cast(inf as float) as timestamp) AS CAST(CAST(inf AS FLOAT) AS TIMESTAMP)#x] ++- OneRowRelation + + +-- !query +select cast(interval '1' year as tinyint) +-- !query analysis +Project [cast(INTERVAL '1' YEAR as tinyint) AS CAST(INTERVAL '1' YEAR AS TINYINT)#x] ++- OneRowRelation + + +-- !query +select cast(interval '-10-2' year to month as smallint) +-- !query analysis +Project [cast(INTERVAL '-10-2' YEAR TO MONTH as smallint) AS CAST(INTERVAL '-10-2' YEAR TO MONTH AS SMALLINT)#x] ++- OneRowRelation + + +-- !query +select cast(interval '1000' month as int) +-- !query analysis +Project [cast(INTERVAL '1000' MONTH as int) AS CAST(INTERVAL '1000' MONTH AS INT)#x] ++- OneRowRelation + + +-- !query +select cast(interval -'10.123456' second as tinyint) +-- !query analysis +Project [cast(INTERVAL '-10.123456' SECOND as tinyint) AS CAST(INTERVAL '-10.123456' SECOND AS TINYINT)#x] ++- OneRowRelation + + +-- !query +select cast(interval '23:59:59' hour to second as smallint) +-- !query analysis +Project [cast(INTERVAL '23:59:59' HOUR TO SECOND as smallint) AS CAST(INTERVAL '23:59:59' HOUR TO SECOND AS SMALLINT)#x] ++- OneRowRelation + + +-- !query +select cast(interval -'1 02:03:04.123' day to second as int) +-- !query analysis +Project [cast(INTERVAL '-1 02:03:04.123' DAY TO SECOND as int) AS CAST(INTERVAL '-1 02:03:04.123' DAY TO SECOND AS INT)#x] ++- OneRowRelation + + +-- !query +select cast(interval '10' day as bigint) +-- !query analysis +Project [cast(INTERVAL '10' DAY as bigint) AS CAST(INTERVAL '10' DAY AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +select cast(interval '-1000' month as tinyint) +-- !query analysis +Project [cast(INTERVAL '-1000' MONTH as tinyint) AS CAST(INTERVAL '-1000' MONTH AS TINYINT)#x] ++- OneRowRelation + + +-- !query +select cast(interval '1000000' second as smallint) +-- !query analysis +Project [cast(INTERVAL '1000000' SECOND as smallint) AS CAST(INTERVAL '1000000' SECOND AS SMALLINT)#x] ++- OneRowRelation + + +-- !query +select cast(1Y as interval year) +-- !query analysis +Project [cast(1 as interval year) AS CAST(1 AS INTERVAL YEAR)#x] ++- OneRowRelation + + +-- !query +select cast(-122S as interval year to month) +-- !query analysis +Project [cast(-122 as interval year to month) AS CAST(-122 AS INTERVAL YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select cast(ym as interval year to month) from values(-122S) as t(ym) +-- !query analysis +Project [cast(ym#x as interval year to month) AS ym#x] ++- SubqueryAlias t + +- LocalRelation [ym#x] + + +-- !query +select cast(1000 as interval month) +-- !query analysis +Project [cast(1000 as interval month) AS CAST(1000 AS INTERVAL MONTH)#x] ++- OneRowRelation + + +-- !query +select cast(-10L as interval second) +-- !query analysis +Project [cast(-10 as interval second) AS CAST(-10 AS INTERVAL SECOND)#x] ++- OneRowRelation + + +-- !query +select cast(100Y as interval hour to second) +-- !query analysis +Project [cast(100 as interval hour to second) AS CAST(100 AS INTERVAL HOUR TO SECOND)#x] ++- OneRowRelation + + +-- !query +select cast(dt as interval hour to second) from values(100Y) as t(dt) +-- !query analysis +Project [cast(dt#x as interval hour to second) AS dt#x] ++- SubqueryAlias t + +- LocalRelation [dt#x] + + +-- !query +select cast(-1000S as interval day to second) +-- !query analysis +Project [cast(-1000 as interval day to second) AS CAST(-1000 AS INTERVAL DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select cast(10 as interval day) +-- !query analysis +Project [cast(10 as interval day) AS CAST(10 AS INTERVAL DAY)#x] ++- OneRowRelation + + +-- !query +select cast(2147483647 as interval year) +-- !query analysis +Project [cast(2147483647 as interval year) AS CAST(2147483647 AS INTERVAL YEAR)#x] ++- OneRowRelation + + +-- !query +select cast(-9223372036854775808L as interval day) +-- !query analysis +Project [cast(-9223372036854775808 as interval day) AS CAST(-9223372036854775808 AS INTERVAL DAY)#x] ++- OneRowRelation + + +-- !query +select cast(interval '-1' year as decimal(10, 0)) +-- !query analysis +Project [cast(INTERVAL '-1' YEAR as decimal(10,0)) AS CAST(INTERVAL '-1' YEAR AS DECIMAL(10,0))#x] ++- OneRowRelation + + +-- !query +select cast(interval '1.000001' second as decimal(10, 6)) +-- !query analysis +Project [cast(INTERVAL '01.000001' SECOND as decimal(10,6)) AS CAST(INTERVAL '01.000001' SECOND AS DECIMAL(10,6))#x] ++- OneRowRelation + + +-- !query +select cast(interval '08:11:10.001' hour to second as decimal(10, 4)) +-- !query analysis +Project [cast(INTERVAL '08:11:10.001' HOUR TO SECOND as decimal(10,4)) AS CAST(INTERVAL '08:11:10.001' HOUR TO SECOND AS DECIMAL(10,4))#x] ++- OneRowRelation + + +-- !query +select cast(interval '1 01:02:03.1' day to second as decimal(8, 1)) +-- !query analysis +Project [cast(INTERVAL '1 01:02:03.1' DAY TO SECOND as decimal(8,1)) AS CAST(INTERVAL '1 01:02:03.1' DAY TO SECOND AS DECIMAL(8,1))#x] ++- OneRowRelation + + +-- !query +select cast(interval '10.123' second as decimal(4, 2)) +-- !query analysis +Project [cast(INTERVAL '10.123' SECOND as decimal(4,2)) AS CAST(INTERVAL '10.123' SECOND AS DECIMAL(4,2))#x] ++- OneRowRelation + + +-- !query +select cast(interval '10.005' second as decimal(4, 2)) +-- !query analysis +Project [cast(INTERVAL '10.005' SECOND as decimal(4,2)) AS CAST(INTERVAL '10.005' SECOND AS DECIMAL(4,2))#x] ++- OneRowRelation + + +-- !query +select cast(interval '10.123' second as decimal(5, 2)) +-- !query analysis +Project [cast(INTERVAL '10.123' SECOND as decimal(5,2)) AS CAST(INTERVAL '10.123' SECOND AS DECIMAL(5,2))#x] ++- OneRowRelation + + +-- !query +select cast(interval '10.123' second as decimal(1, 0)) +-- !query analysis +Project [cast(INTERVAL '10.123' SECOND as decimal(1,0)) AS CAST(INTERVAL '10.123' SECOND AS DECIMAL(1,0))#x] ++- OneRowRelation + + +-- !query +select cast(10.123456BD as interval day to second) +-- !query analysis +Project [cast(10.123456 as interval day to second) AS CAST(10.123456 AS INTERVAL DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select cast(80.654321BD as interval hour to minute) +-- !query analysis +Project [cast(80.654321 as interval hour to minute) AS CAST(80.654321 AS INTERVAL HOUR TO MINUTE)#x] ++- OneRowRelation + + +-- !query +select cast(-10.123456BD as interval year to month) +-- !query analysis +Project [cast(-10.123456 as interval year to month) AS CAST(-10.123456 AS INTERVAL YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select cast(10.654321BD as interval month) +-- !query analysis +Project [cast(10.654321 as interval month) AS CAST(10.654321 AS INTERVAL MONTH)#x] ++- OneRowRelation 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/ansi/conditional-functions.sql.out new file mode 100644 index 0000000000000..0bd8648698b48 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/conditional-functions.sql.out @@ -0,0 +1,104 @@ +-- 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 +DROP TABLE conditional_t +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`conditional_t`, false, false, false 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/ansi/date.sql.out new file mode 100644 index 0000000000000..3929e02dd1b46 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/date.sql.out @@ -0,0 +1,867 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view date_view as select '2011-11-11' date_str, '1' int_str +-- !query analysis +CreateViewCommand `date_view`, select '2011-11-11' date_str, '1' int_str, false, false, LocalTempView, true + +- Project [2011-11-11 AS date_str#x, 1 AS int_str#x] + +- OneRowRelation + + +-- !query +select date '2019-01-01\t' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2020-01-01中文' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2020-01-01中文'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "date '2020-01-01中文'" + } ] +} + + +-- !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] ++- OneRowRelation + + +-- !query +select make_date(2000, 13, 1) +-- !query analysis +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, true) AS make_date(2000, 1, 33)#x] ++- OneRowRelation + + +-- !query +select date'015' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'015'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "date'015'" + } ] +} + + +-- !query +select date'2021-4294967297-11' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-4294967297-11'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "date'2021-4294967297-11'" + } ] +} + + +-- !query +select current_date = current_date +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select current_date() = current_date() +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select curdate(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "0", + "functionName" : "`curdate`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "curdate(1)" + } ] +} + + +-- !query +select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null) +-- !query analysis +Project [date_from_unix_date(0) AS date_from_unix_date(0)#x, date_from_unix_date(1000) AS date_from_unix_date(1000)#x, date_from_unix_date(cast(null as int)) AS date_from_unix_date(NULL)#x] ++- OneRowRelation + + +-- !query +select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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)) AS to_date(NULL)#x, to_date(2016-12-31, None, Some(America/Los_Angeles)) AS to_date(2016-12-31)#x, to_date(2016-12-31, Some(yyyy-MM-dd), Some(America/Los_Angeles)) 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)) 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)) AS to_date(02-29, MM-dd)#x] ++- OneRowRelation + + +-- !query +select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), + dayofweek('1582-10-15 13:10:15'), dayofweek(timestamp_ltz'1582-10-15 13:10:15'), dayofweek(timestamp_ntz'1582-10-15 13:10:15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), + weekday('1582-10-15 13:10:15'), weekday(timestamp_ltz'1582-10-15 13:10:15'), weekday(timestamp_ntz'1582-10-15 13:10:15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select year('1500-01-01'), year('1582-10-15 13:10:15'), year(timestamp_ltz'1582-10-15 13:10:15'), year(timestamp_ntz'1582-10-15 13:10:15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select month('1500-01-01'), month('1582-10-15 13:10:15'), month(timestamp_ltz'1582-10-15 13:10:15'), month(timestamp_ntz'1582-10-15 13:10:15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(timestamp_ltz'1582-10-15 13:10:15'), dayOfYear(timestamp_ntz'1582-10-15 13:10:15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon") +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- OneRowRelation + + +-- !query +select next_day("xx", "Mon") +-- !query analysis +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, true) 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] ++- OneRowRelation + + +-- !query +select date_add(date'2011-11-11', 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- OneRowRelation + + +-- !query +select date_add('2011-11-11', 1Y) +-- !query analysis +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', 1S) +-- !query analysis +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', 1L) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "date_add('2011-11-11', 1L)" + } ] +} + + +-- !query +select date_add('2011-11-11', 1.0) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_add('2011-11-11', 1.0)" + } ] +} + + +-- !query +select date_add('2011-11-11', 1E1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_add('2011-11-11', 1E1)" + } ] +} + + +-- !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] ++- 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 + + +-- !query +select date_add(null, 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_add(date'2011-11-11', null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_add(timestamp_ltz'2011-11-11 12:12:12', 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_add(timestamp_ntz'2011-11-11 12:12:12', 1) +-- !query analysis +Project [date_add(cast(2011-11-11 12:12:12 as date), 1) AS date_add(TIMESTAMP_NTZ '2011-11-11 12:12:12', 1)#x] ++- OneRowRelation + + +-- !query +select date_sub(date'2011-11-11', 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_sub('2011-11-11', 1) +-- !query analysis +Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x] ++- OneRowRelation + + +-- !query +select date_sub('2011-11-11', 1Y) +-- !query analysis +Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x] ++- OneRowRelation + + +-- !query +select date_sub('2011-11-11', 1S) +-- !query analysis +Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x] ++- OneRowRelation + + +-- !query +select date_sub('2011-11-11', 1L) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "date_sub('2011-11-11', 1L)" + } ] +} + + +-- !query +select date_sub('2011-11-11', 1.0) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_sub('2011-11-11', 1.0)" + } ] +} + + +-- !query +select date_sub('2011-11-11', 1E1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_sub('2011-11-11', 1E1)" + } ] +} + + +-- !query +select date_sub(date'2011-11-11', '1') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_sub(date'2011-11-11', '1.2') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_sub(null, 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_sub(date'2011-11-11', null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_sub(timestamp_ltz'2011-11-11 12:12:12', 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_sub(timestamp_ntz'2011-11-11 12:12:12', 1) +-- !query analysis +Project [date_sub(cast(2011-11-11 12:12:12 as date), 1) AS date_sub(TIMESTAMP_NTZ '2011-11-11 12:12:12', 1)#x] ++- OneRowRelation + + +-- !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 + + +-- !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 + + +-- !query +select date_add(date_str, 1) from date_view +-- !query analysis +Project [date_add(cast(date_str#x as date), 1) AS date_add(date_str, 1)#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(date_str, 1) from date_view +-- !query analysis +Project [date_sub(cast(date_str#x as date), 1) AS date_sub(date_str, 1)#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 '2011-11-11' + 1E1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "date '2011-11-11' + 1E1" + } ] +} + + +-- !query +select date '2001-09-28' + 7Y +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select 7S + date '2001-09-28' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2001-10-01' - 7 +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2001-10-01' - date '2001-09-28' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2001-10-01' - '2001-09-28' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select '2001-10-01' - date '2001-09-28' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2001-09-28' - null +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select null - date '2019-10-06' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_str - date '2001-09-28' from date_view +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2001-09-28' - date_str from date_view +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date'2011-11-11' + '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"DATE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "date'2011-11-11' + '1'" + } ] +} + + +-- !query +select '1' + date'2011-11-11' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"DATE '2011-11-11'\"", + "inputType" : "\"DATE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(1, DATE '2011-11-11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' + date'2011-11-11'" + } ] +} + + +-- !query +select date'2011-11-11' + null +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select null + date'2011-11-11' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2012-01-01' - interval '2-2' year to month, + date '2011-11-11' - interval '2' day, + date '2012-01-01' + interval '-2-2' year to month, + date '2011-11-11' + interval '-2' month, + - interval '2-2' year to month + date '2012-01-01', + interval '-2' day + date '2011-11-11' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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)) AS to_date(26/October/2015, dd/MMMMM/yyyy)#x] ++- OneRowRelation + + +-- !query +select from_json('{"d":"26/October/2015"}', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_json(StructField(d,DateType,true), (dateFormat,dd/MMMMM/yyyy), {"d":"26/October/2015"}, Some(America/Los_Angeles)) AS from_json({"d":"26/October/2015"})#x] ++- OneRowRelation + + +-- !query +select from_csv('26/October/2015', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_csv(StructField(d,DateType,true), (dateFormat,dd/MMMMM/yyyy), 26/October/2015, Some(America/Los_Angeles), None) AS from_csv(26/October/2015)#x] ++- OneRowRelation + + +-- !query +select dateadd(MICROSECOND, 1001, timestamp'2022-02-25 01:02:03.123') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(MILLISECOND, -1, timestamp'2022-02-25 01:02:03.456') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(SECOND, 58, timestamp'2022-02-25 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(MINUTE, -100, date'2022-02-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(HOUR, -1, timestamp'2022-02-25 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(DAY, 367, date'2022-02-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(WEEK, -4, timestamp'2022-02-25 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(MONTH, -1, timestamp'2022-02-25 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(QUARTER, 5, date'2022-02-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(YEAR, 1, date'2022-02-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(MICROSECOND, timestamp'2022-02-25 01:02:03.123', timestamp'2022-02-25 01:02:03.124001') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(MILLISECOND, timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(SECOND, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 01:03:01') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(MINUTE, date'2022-02-25', timestamp'2022-02-24 22:20:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(HOUR, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 00:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(DAY, date'2022-02-25', timestamp'2023-02-27 00:00:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(WEEK, timestamp'2022-02-25 01:02:03', timestamp'2022-01-28 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(MONTH, timestamp'2022-02-25 01:02:03', timestamp'2022-01-25 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(QUARTER, date'2022-02-25', date'2023-05-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(YEAR, date'2022-02-25', date'2023-02-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] 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/ansi/datetime-parsing-invalid.sql.out new file mode 100644 index 0000000000000..0c7f2cb193da9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/datetime-parsing-invalid.sql.out @@ -0,0 +1,195 @@ +-- Automatically generated by SQLQueryTestSuite +-- !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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD')) +-- !query analysis +Project [from_csv(StructField(date,DateType,true), (dateFormat,yyyy-DDD), 2018-366, Some(America/Los_Angeles), None) AS from_csv(2018-366)#x] ++- OneRowRelation + + +-- !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)) 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)) 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select cast("Unparseable" as timestamp) +-- !query analysis +Project [cast(Unparseable as timestamp) AS CAST(Unparseable AS TIMESTAMP)#x] ++- OneRowRelation + + +-- !query +select cast("Unparseable" as date) +-- !query analysis +Project [cast(Unparseable as date) AS CAST(Unparseable AS DATE)#x] ++- 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/ansi/datetime-special.sql.out new file mode 100644 index 0000000000000..6768297fd8116 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/datetime-special.sql.out @@ -0,0 +1,25 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select date'999999-03-18', date'-0001-1-28', date'0015' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- OneRowRelation + + +-- !query +select timestamp'-1969-12-31 16:00:00', timestamp'-0015-03-18 16:00:00', timestamp'-000001', timestamp'99999-03-18T12:03:17' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- 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/ansi/decimalArithmeticOperations.sql.out new file mode 100644 index 0000000000000..8abc9148de054 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/decimalArithmeticOperations.sql.out @@ -0,0 +1,169 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create table decimals_test(id int, a decimal(38,18), b decimal(38,18)) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`decimals_test`, false + + +-- !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 analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/decimals_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/decimals_test], Append, `spark_catalog`.`default`.`decimals_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/decimals_test), [id, a, b] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,18)) AS a#x, cast(col3#x as decimal(38,18)) AS b#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !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 (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 123456789123456789.1234567890 * 1.123456789123456789 +-- !query analysis +Project [(123456789123456789.1234567890 * 1.123456789123456789) AS (123456789123456789.1234567890 * 1.123456789123456789)#x] ++- OneRowRelation + + +-- !query +select 123456789123456789.1234567890 * 1.123456789123456789 +-- !query analysis +Project [(123456789123456789.1234567890 * 1.123456789123456789) AS (123456789123456789.1234567890 * 1.123456789123456789)#x] ++- 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 1.0123456789012345678901234567890123456e36BD / 0.1 +-- !query analysis +Project [(1012345678901234567890123456789012345.6 / 0.1) AS (1012345678901234567890123456789012345.6 / 0.1)#x] ++- OneRowRelation + + +-- !query +select 1.0123456789012345678901234567890123456e35BD / 1.0 +-- !query analysis +Project [(101234567890123456789012345678901234.56 / 1.0) AS (101234567890123456789012345678901234.56 / 1.0)#x] ++- OneRowRelation + + +-- !query +select 1.0123456789012345678901234567890123456e34BD / 1.0 +-- !query analysis +Project [(10123456789012345678901234567890123.456 / 1.0) AS (10123456789012345678901234567890123.456 / 1.0)#x] ++- OneRowRelation + + +-- !query +select 1.0123456789012345678901234567890123456e33BD / 1.0 +-- !query analysis +Project [(1012345678901234567890123456789012.3456 / 1.0) AS (1012345678901234567890123456789012.3456 / 1.0)#x] ++- OneRowRelation + + +-- !query +select 1.0123456789012345678901234567890123456e32BD / 1.0 +-- !query analysis +Project [(101234567890123456789012345678901.23456 / 1.0) AS (101234567890123456789012345678901.23456 / 1.0)#x] ++- OneRowRelation + + +-- !query +select 1.0123456789012345678901234567890123456e31BD / 1.0 +-- !query analysis +Project [(10123456789012345678901234567890.123456 / 1.0) AS (10123456789012345678901234567890.123456 / 1.0)#x] ++- OneRowRelation + + +-- !query +select 1.0123456789012345678901234567890123456e31BD / 0.1 +-- !query analysis +Project [(10123456789012345678901234567890.123456 / 0.1) AS (10123456789012345678901234567890.123456 / 0.1)#x] ++- OneRowRelation + + +-- !query +select 1.0123456789012345678901234567890123456e31BD / 10.0 +-- !query analysis +Project [(10123456789012345678901234567890.123456 / 10.0) AS (10123456789012345678901234567890.123456 / 10.0)#x] ++- OneRowRelation + + +-- !query +drop table decimals_test +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`decimals_test`, false, false, false 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/ansi/double-quoted-identifiers-disabled.sql.out new file mode 100644 index 0000000000000..501eec2c963ac --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-disabled.sql.out @@ -0,0 +1,393 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT 1 FROM "not_exist" +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +USE SCHEMA "not_exist" +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +ALTER TABLE "not_exist" ADD COLUMN not_exist int +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +ALTER TABLE not_exist ADD COLUMN "not_exist" int +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT 1 AS "not_exist" FROM not_exist +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT 1 FROM not_exist AS X("hello") +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"hello\"'", + "hint" : "" + } +} + + +-- !query +SELECT "not_exist"() +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT "not_exist".not_exist() +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT 1 FROM `hello` +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`hello`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 21, + "fragment" : "`hello`" + } ] +} + + +-- !query +USE SCHEMA `not_exist` +-- !query analysis +org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +{ + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42704", + "messageParameters" : { + "schemaName" : "`not_exist`" + } +} + + +-- !query +ALTER TABLE `not_exist` ADD COLUMN not_exist int +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "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.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "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.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 40, + "fragment" : "`not_exist`" + } ] +} + + +-- !query +SELECT 1 FROM not_exist AS X(`hello`) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 37, + "fragment" : "not_exist AS X(`hello`)" + } ] +} + + +-- !query +SELECT `not_exist`() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "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.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "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()" + } ] +} + + +-- !query +SELECT "hello" +-- !query analysis +Project [hello AS hello#x] ++- OneRowRelation + + +-- !query +CREATE TEMPORARY VIEW v(c1 COMMENT "hello") AS SELECT 1 +-- !query analysis +CreateViewCommand `v`, [(c1,Some(hello))], SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +DROP VIEW v +-- !query analysis +DropTempViewCommand v + + +-- !query +SELECT INTERVAL "1" YEAR +-- !query analysis +Project [INTERVAL '1' YEAR AS INTERVAL '1' YEAR#x] ++- OneRowRelation + + +-- !query +SELECT 'hello' +-- !query analysis +Project [hello AS hello#x] ++- OneRowRelation + + +-- !query +CREATE TEMPORARY VIEW v(c1 COMMENT 'hello') AS SELECT 1 +-- !query analysis +CreateViewCommand `v`, [(c1,Some(hello))], SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +DROP VIEW v +-- !query analysis +DropTempViewCommand v + + +-- !query +SELECT INTERVAL '1' YEAR +-- !query analysis +Project [INTERVAL '1' YEAR AS INTERVAL '1' YEAR#x] ++- OneRowRelation + + +-- !query +CREATE SCHEMA "myschema" +-- !query analysis +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 +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 +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"a2\"'", + "hint" : "" + } +} + + +-- !query +DROP TABLE "myview" +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myview\"'", + "hint" : "" + } +} + + +-- !query +DROP SCHEMA "myschema" +-- !query analysis +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/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out new file mode 100644 index 0000000000000..327cbf009b555 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out @@ -0,0 +1,450 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT 1 FROM "not_exist" +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "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.analysis.NoSuchDatabaseException +{ + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42704", + "messageParameters" : { + "schemaName" : "`not_exist`" + } +} + + +-- !query +ALTER TABLE "not_exist" ADD COLUMN not_exist int +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "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.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "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.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "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.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 37, + "fragment" : "not_exist AS X(\"hello\")" + } ] +} + + +-- !query +SELECT "not_exist"() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "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.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "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()" + } ] +} + + +-- !query +SELECT 1 FROM `hello` +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`hello`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 21, + "fragment" : "`hello`" + } ] +} + + +-- !query +USE SCHEMA `not_exist` +-- !query analysis +org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +{ + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42704", + "messageParameters" : { + "schemaName" : "`not_exist`" + } +} + + +-- !query +ALTER TABLE `not_exist` ADD COLUMN not_exist int +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "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.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "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.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 40, + "fragment" : "`not_exist`" + } ] +} + + +-- !query +SELECT 1 FROM not_exist AS X(`hello`) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 37, + "fragment" : "not_exist AS X(`hello`)" + } ] +} + + +-- !query +SELECT `not_exist`() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "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.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "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()" + } ] +} + + +-- !query +SELECT "hello" +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"hello\"'", + "hint" : "" + } +} + + +-- !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`" + } +} + + +-- !query +SELECT INTERVAL "1" YEAR +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"1\"'", + "hint" : "" + } +} + + +-- !query +SELECT 'hello' +-- !query analysis +Project [hello AS hello#x] ++- OneRowRelation + + +-- !query +CREATE TEMPORARY VIEW v(c1 COMMENT 'hello') AS SELECT 1 +-- !query analysis +CreateViewCommand `v`, [(c1,Some(hello))], SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +DROP VIEW v +-- !query analysis +DropTempViewCommand v + + +-- !query +SELECT INTERVAL '1' YEAR +-- !query analysis +Project [INTERVAL '1' YEAR AS INTERVAL '1' YEAR#x] ++- OneRowRelation + + +-- !query +CREATE SCHEMA "myschema" +-- !query analysis +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 +CreateViewCommand `myview`, [(c1,None)], WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v", false, false, LocalTempView, true + +- Project [a#x] + +- SubqueryAlias v + +- Project [1#x AS a#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] + + +-- !query +DROP TABLE "myview" +-- !query analysis +DropTempViewCommand myview + + +-- !query +DROP SCHEMA "myschema" +-- !query analysis +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [myschema] 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/ansi/higher-order-functions.sql.out new file mode 100644 index 0000000000000..ac107e3e8c381 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/higher-order-functions.sql.out @@ -0,0 +1,351 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create or replace temporary view nested as values + (1, array(32, 97), array(array(12, 99), array(123, 42), array(1))), + (2, array(77, -76), array(array(6, 96, 65), array(-1, -2))), + (3, array(12), array(array(17))) + as t(x, ys, zs) +-- !query analysis +CreateViewCommand `nested`, values + (1, array(32, 97), array(array(12, 99), array(123, 42), array(1))), + (2, array(77, -76), array(array(6, 96, 65), array(-1, -2))), + (3, array(12), array(array(17))) + as t(x, ys, zs), false, true, LocalTempView, true + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select upper(x -> x) as v +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2306", + "messageParameters" : { + "class" : "org.apache.spark.sql.catalyst.expressions.Upper" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "upper(x -> x)" + } ] +} + + +-- !query +select transform(zs, z -> z) as v from nested +-- !query analysis +Project [transform(zs#x, lambdafunction(lambda z#x, 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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select transform(ys, y -> y * y) as v from nested +-- !query analysis +Project [transform(ys#x, lambdafunction((lambda y#x * lambda y#x), lambda y#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select transform(ys, (y, i) -> y + i) as v from nested +-- !query analysis +Project [transform(ys#x, lambdafunction((lambda y#x + lambda i#x), lambda y#x, lambda i#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select transform(zs, z -> concat(ys, z)) as v from nested +-- !query analysis +Project [transform(zs#x, lambdafunction(concat(ys#x, lambda z#x), 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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select transform(ys, 0) as v from nested +-- !query analysis +Project [transform(ys#x, lambdafunction(0, lambda col0#x, true)) 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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select transform(cast(null as array), x -> x + 1) as v +-- !query analysis +Project [transform(cast(null as array), lambdafunction((lambda x#x + 1), lambda x#x, false)) AS v#x] ++- OneRowRelation + + +-- !query +select filter(ys, y -> y > 30) as v from nested +-- !query analysis +Project [filter(ys#x, lambdafunction((lambda y#x > 30), lambda y#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select filter(cast(null as array), y -> true) as v +-- !query analysis +Project [filter(cast(null as array), lambdafunction(true, lambda y#x, false)) AS v#x] ++- OneRowRelation + + +-- !query +select transform(zs, z -> filter(z, zz -> zz > 50)) as v from nested +-- !query analysis +Project [transform(zs#x, lambdafunction(filter(lambda z#x, lambdafunction((lambda zz#x > 50), lambda zz#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select aggregate(ys, 0, (y, a) -> y + a + x) as v from nested +-- !query analysis +Project [aggregate(ys#x, 0, lambdafunction(((lambda y#x + lambda a#x) + x#x), lambda y#x, lambda a#x, false), lambdafunction(lambda id#x, lambda id#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select aggregate(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested +-- !query analysis +Project [aggregate(ys#x, named_struct(sum, 0, n, 0), lambdafunction(named_struct(col1, (lambda acc#x.sum + lambda x#x), col2, (lambda acc#x.n + 1)), lambda acc#x, lambda x#x, false), lambdafunction((cast(lambda acc#x.sum as double) / cast(lambda acc#x.n as double)), lambda acc#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !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] ++- 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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select aggregate(cast(null as array), 0, (a, y) -> a + y + 1, a -> a + 2) as v +-- !query analysis +Project [aggregate(cast(null as array), 0, lambdafunction(((lambda a#x + lambda y#x) + 1), lambda a#x, lambda y#x, false), lambdafunction((lambda a#x + 2), lambda a#x, false)) AS v#x] ++- OneRowRelation + + +-- !query +select reduce(ys, 0, (y, a) -> y + a + x) as v from nested +-- !query analysis +Project [reduce(ys#x, 0, lambdafunction(((lambda y#x + lambda a#x) + x#x), lambda y#x, lambda a#x, false), lambdafunction(lambda id#x, lambda id#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select reduce(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested +-- !query analysis +Project [reduce(ys#x, named_struct(sum, 0, n, 0), lambdafunction(named_struct(col1, (lambda acc#x.sum + lambda x#x), col2, (lambda acc#x.n + 1)), lambda acc#x, lambda x#x, false), lambdafunction((cast(lambda acc#x.sum as double) / cast(lambda acc#x.n as double)), lambda acc#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !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] ++- 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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select reduce(cast(null as array), 0, (a, y) -> a + y + 1, a -> a + 2) as v +-- !query analysis +Project [reduce(cast(null as array), 0, lambdafunction(((lambda a#x + lambda y#x) + 1), lambda a#x, lambda y#x, false), lambdafunction((lambda a#x + 2), lambda a#x, false)) AS v#x] ++- OneRowRelation + + +-- !query +select exists(ys, y -> y > 30) as v from nested +-- !query analysis +Project [exists(ys#x, lambdafunction((lambda y#x > 30), lambda y#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select exists(cast(null as array), y -> y > 30) as v +-- !query analysis +Project [exists(cast(null as array), lambdafunction((lambda y#x > 30), lambda y#x, false)) AS v#x] ++- OneRowRelation + + +-- !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] ++- 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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select zip_with(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) -> concat(x, y)) as v +-- !query analysis +Project [zip_with(array(a, b, c), array(d, e, f), lambdafunction(concat(lambda x#x, lambda y#x), lambda x#x, lambda y#x, false)) AS v#x] ++- OneRowRelation + + +-- !query +select zip_with(array('a'), array('d', null, 'f'), (x, y) -> coalesce(x, y)) as v +-- !query analysis +Project [zip_with(array(a), array(d, cast(null as string), f), lambdafunction(coalesce(lambda x#x, lambda y#x), lambda x#x, lambda y#x, false)) AS v#x] ++- OneRowRelation + + +-- !query +create or replace temporary view nested as values + (1, map(1, 1, 2, 2, 3, 3)), + (2, map(4, 4, 5, 5, 6, 6)) + as t(x, ys) +-- !query analysis +CreateViewCommand `nested`, values + (1, map(1, 1, 2, 2, 3, 3)), + (2, map(4, 4, 5, 5, 6, 6)) + as t(x, ys), false, true, LocalTempView, true + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x] + + +-- !query +select transform_keys(ys, (k, v) -> k) as v from nested +-- !query analysis +Project [transform_keys(ys#x, lambdafunction(lambda k#x, lambda k#x, lambda v#x, false)) AS v#x] ++- SubqueryAlias nested + +- View (`nested`, [x#x,ys#x]) + +- Project [cast(x#x as int) AS x#x, cast(ys#x as map) AS ys#x] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x] + + +-- !query +select transform_keys(ys, (k, v) -> k + 1) as v from nested +-- !query analysis +Project [transform_keys(ys#x, lambdafunction((lambda k#x + 1), lambda k#x, lambda v#x, false)) AS v#x] ++- SubqueryAlias nested + +- View (`nested`, [x#x,ys#x]) + +- Project [cast(x#x as int) AS x#x, cast(ys#x as map) AS ys#x] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x] + + +-- !query +select transform_keys(ys, (k, v) -> k + v) as v from nested +-- !query analysis +Project [transform_keys(ys#x, lambdafunction((lambda k#x + lambda v#x), lambda k#x, lambda v#x, false)) AS v#x] ++- SubqueryAlias nested + +- View (`nested`, [x#x,ys#x]) + +- Project [cast(x#x as int) AS x#x, cast(ys#x as map) AS ys#x] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x] + + +-- !query +select transform_values(ys, (k, v) -> v) as v from nested +-- !query analysis +Project [transform_values(ys#x, lambdafunction(lambda v#x, lambda k#x, lambda v#x, false)) AS v#x] ++- SubqueryAlias nested + +- View (`nested`, [x#x,ys#x]) + +- Project [cast(x#x as int) AS x#x, cast(ys#x as map) AS ys#x] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x] + + +-- !query +select transform_values(ys, (k, v) -> v + 1) as v from nested +-- !query analysis +Project [transform_values(ys#x, lambdafunction((lambda v#x + 1), lambda k#x, lambda v#x, false)) AS v#x] ++- SubqueryAlias nested + +- View (`nested`, [x#x,ys#x]) + +- Project [cast(x#x as int) AS x#x, cast(ys#x as map) AS ys#x] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x] + + +-- !query +select transform_values(ys, (k, v) -> k + v) as v from nested +-- !query analysis +Project [transform_values(ys#x, lambdafunction((lambda k#x + lambda v#x), lambda k#x, lambda v#x, false)) AS v#x] ++- SubqueryAlias nested + +- View (`nested`, [x#x,ys#x]) + +- Project [cast(x#x as int) AS x#x, cast(ys#x as map) AS ys#x] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x] + + +-- !query +select transform(ys, all -> all * all) as v from values (array(32, 97)) as t(ys) +-- !query analysis +Project [transform(ys#x, lambdafunction((lambda all#x * lambda all#x), lambda all#x, false)) AS v#x] ++- SubqueryAlias t + +- LocalRelation [ys#x] + + +-- !query +select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t(ys) +-- !query analysis +Project [transform(ys#x, lambdafunction((lambda all#x + lambda i#x), lambda all#x, lambda i#x, false)) AS v#x] ++- SubqueryAlias t + +- LocalRelation [ys#x] + + +-- !query +select aggregate(split('abcdefgh',''), array(array('')), (acc, x) -> array(array(x))) +-- !query analysis +Project [aggregate(split(abcdefgh, , -1), array(array()), lambdafunction(array(array(lambda x#x)), lambda acc#x, lambda x#x, false), lambdafunction(lambda id#x, lambda id#x, false)) AS aggregate(split(abcdefgh, , -1), array(array()), lambdafunction(array(array(namedlambdavariable())), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable()))#x] ++- OneRowRelation 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/ansi/interval.sql.out new file mode 100644 index 0000000000000..19c77c8de9071 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/interval.sql.out @@ -0,0 +1,3183 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select interval 4 month 2 weeks 3 microseconds * 1.5 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 4 month 2 weeks 3 microseconds" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "interval 4 month 2 weeks 3 microseconds" + } ] +} + + +-- !query +select interval 2 years 4 months +-- !query analysis +Project [INTERVAL '2-4' YEAR TO MONTH AS INTERVAL '2-4' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval 2 weeks 3 microseconds * 1.5 +-- !query analysis +Project [(INTERVAL '14 00:00:00.000003' DAY TO SECOND * 1.5) AS (INTERVAL '14 00:00:00.000003' DAY TO SECOND * 1.5)#x] ++- OneRowRelation + + +-- !query +select (timestamp'2019-10-15' - timestamp'2019-10-14') / 1.5 +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select interval 2147483647 month * 2 +-- !query analysis +Project [(INTERVAL '2147483647' MONTH * 2) AS (INTERVAL '2147483647' MONTH * 2)#x] ++- OneRowRelation + + +-- !query +select interval 2147483647 month / 0.5 +-- !query analysis +Project [(INTERVAL '2147483647' MONTH / 0.5) AS (INTERVAL '2147483647' MONTH / 0.5)#x] ++- OneRowRelation + + +-- !query +select interval 2147483647 day * 2 +-- !query analysis +java.lang.ArithmeticException +long overflow + + +-- !query +select interval 2147483647 day / 0.5 +-- !query analysis +java.lang.ArithmeticException +long overflow + + +-- !query +select interval 2 second * '2' +-- !query analysis +Project [(INTERVAL '02' SECOND * cast(2 as double)) AS (INTERVAL '02' SECOND * 2)#x] ++- OneRowRelation + + +-- !query +select interval 2 second / '2' +-- !query analysis +Project [(INTERVAL '02' SECOND / cast(2 as double)) AS (INTERVAL '02' SECOND / 2)#x] ++- OneRowRelation + + +-- !query +select interval 2 year * '2' +-- !query analysis +Project [(INTERVAL '2' YEAR * cast(2 as double)) AS (INTERVAL '2' YEAR * 2)#x] ++- OneRowRelation + + +-- !query +select interval 2 year / '2' +-- !query analysis +Project [(INTERVAL '2' YEAR / cast(2 as double)) AS (INTERVAL '2' YEAR / 2)#x] ++- OneRowRelation + + +-- !query +select interval 2 second * 'a' +-- !query analysis +Project [(INTERVAL '02' SECOND * cast(a as double)) AS (INTERVAL '02' SECOND * a)#x] ++- OneRowRelation + + +-- !query +select interval 2 second / 'a' +-- !query analysis +Project [(INTERVAL '02' SECOND / cast(a as double)) AS (INTERVAL '02' SECOND / a)#x] ++- OneRowRelation + + +-- !query +select interval 2 year * 'a' +-- !query analysis +Project [(INTERVAL '2' YEAR * cast(a as double)) AS (INTERVAL '2' YEAR * a)#x] ++- OneRowRelation + + +-- !query +select interval 2 year / 'a' +-- !query analysis +Project [(INTERVAL '2' YEAR / cast(a as double)) AS (INTERVAL '2' YEAR / a)#x] ++- OneRowRelation + + +-- !query +select '2' * interval 2 second +-- !query analysis +Project [(INTERVAL '02' SECOND * cast(2 as double)) AS (INTERVAL '02' SECOND * 2)#x] ++- OneRowRelation + + +-- !query +select '2' * interval 2 year +-- !query analysis +Project [(INTERVAL '2' YEAR * cast(2 as double)) AS (INTERVAL '2' YEAR * 2)#x] ++- OneRowRelation + + +-- !query +select 'a' * interval 2 second +-- !query analysis +Project [(INTERVAL '02' SECOND * cast(a as double)) AS (INTERVAL '02' SECOND * a)#x] ++- OneRowRelation + + +-- !query +select 'a' * interval 2 year +-- !query analysis +Project [(INTERVAL '2' YEAR * cast(a as double)) AS (INTERVAL '2' YEAR * a)#x] ++- OneRowRelation + + +-- !query +select '2' / interval 2 second +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL SECOND\"", + "sqlExpr" : "\"(2 / INTERVAL '02' SECOND)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'2' / interval 2 second" + } ] +} + + +-- !query +select '2' / interval 2 year +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "'2' / interval 2 year" + } ] +} + + +-- !query +select interval '2 seconds' / 0 +-- !query analysis +Project [(INTERVAL '02' SECOND / 0) AS (INTERVAL '02' SECOND / 0)#x] ++- OneRowRelation + + +-- !query +select interval '2 seconds' / null +-- !query analysis +Project [(INTERVAL '02' SECOND / cast(null as double)) AS (INTERVAL '02' SECOND / NULL)#x] ++- OneRowRelation + + +-- !query +select interval '2 seconds' * null +-- !query analysis +Project [(INTERVAL '02' SECOND * cast(null as double)) AS (INTERVAL '02' SECOND * NULL)#x] ++- OneRowRelation + + +-- !query +select null * interval '2 seconds' +-- !query analysis +Project [(INTERVAL '02' SECOND * cast(null as double)) AS (INTERVAL '02' SECOND * NULL)#x] ++- OneRowRelation + + +-- !query +select interval '2' year / 0 +-- !query analysis +Project [(INTERVAL '2' YEAR / 0) AS (INTERVAL '2' YEAR / 0)#x] ++- OneRowRelation + + +-- !query +select interval '2' year / null +-- !query analysis +Project [(INTERVAL '2' YEAR / cast(null as double)) AS (INTERVAL '2' YEAR / NULL)#x] ++- OneRowRelation + + +-- !query +select interval '2' year * null +-- !query analysis +Project [(INTERVAL '2' YEAR * cast(null as double)) AS (INTERVAL '2' YEAR * NULL)#x] ++- OneRowRelation + + +-- !query +select null * interval '2' year +-- !query analysis +Project [(INTERVAL '2' YEAR * cast(null as double)) AS (INTERVAL '2' YEAR * NULL)#x] ++- OneRowRelation + + +-- !query +select 2 / interval '2' year +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "2 / interval '2' year" + } ] +} + + +-- !query +select 2 / interval '2' hour +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"INTERVAL HOUR\"", + "sqlExpr" : "\"(2 / INTERVAL '02' HOUR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "2 / interval '2' hour" + } ] +} + + +-- !query +select null / interval '2' year +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(NULL / INTERVAL '2' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "null / interval '2' year" + } ] +} + + +-- !query +select null / interval '2' hour +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"INTERVAL HOUR\"", + "sqlExpr" : "\"(NULL / INTERVAL '02' HOUR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "null / interval '2' hour" + } ] +} + + +-- !query +select -interval '-1 month 1 day -1 second' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '-1 month 1 day -1 second'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 43, + "fragment" : "interval '-1 month 1 day -1 second'" + } ] +} + + +-- !query +select -interval '-1 year 1 month' +-- !query analysis +Project [-INTERVAL '-0-11' YEAR TO MONTH AS (- INTERVAL '-0-11' YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select -interval '-1 day 1 hour -1 minute 1 second' +-- !query analysis +Project [-INTERVAL '-0 23:00:59' DAY TO SECOND AS (- INTERVAL '-0 23:00:59' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select -interval -1 month 1 day -1 second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval -1 month 1 day -1 second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 41, + "fragment" : "interval -1 month 1 day -1 second" + } ] +} + + +-- !query +select -interval -1 year 1 month +-- !query analysis +Project [-INTERVAL '-0-11' YEAR TO MONTH AS (- INTERVAL '-0-11' YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select -interval -1 day 1 hour -1 minute 1 second +-- !query analysis +Project [-INTERVAL '-0 23:00:59' DAY TO SECOND AS (- INTERVAL '-0 23:00:59' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select +interval '-1 month 1 day -1 second' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '-1 month 1 day -1 second'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 43, + "fragment" : "interval '-1 month 1 day -1 second'" + } ] +} + + +-- !query +select +interval '-1 year 1 month' +-- !query analysis +Project [positive(INTERVAL '-0-11' YEAR TO MONTH) AS (+ INTERVAL '-0-11' YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select +interval '-1 day 1 hour -1 minute 1 second' +-- !query analysis +Project [positive(INTERVAL '-0 23:00:59' DAY TO SECOND) AS (+ INTERVAL '-0 23:00:59' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select +interval -1 month 1 day -1 second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval -1 month 1 day -1 second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 41, + "fragment" : "interval -1 month 1 day -1 second" + } ] +} + + +-- !query +select +interval -1 year 1 month +-- !query analysis +Project [positive(INTERVAL '-0-11' YEAR TO MONTH) AS (+ INTERVAL '-0-11' YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select +interval -1 day 1 hour -1 minute 1 second +-- !query analysis +Project [positive(INTERVAL '-0 23:00:59' DAY TO SECOND) AS (+ INTERVAL '-0 23:00:59' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select interval -'1-1' year to month +-- !query analysis +Project [INTERVAL '-1-1' YEAR TO MONTH AS INTERVAL '-1-1' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval -'-1-1' year to month +-- !query analysis +Project [INTERVAL '1-1' YEAR TO MONTH AS INTERVAL '1-1' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval +'-1-1' year to month +-- !query analysis +Project [INTERVAL '-1-1' YEAR TO MONTH AS INTERVAL '-1-1' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval - '1 2:3:4.001' day to second +-- !query analysis +Project [INTERVAL '-1 02:03:04.001' DAY TO SECOND AS INTERVAL '-1 02:03:04.001' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval +'1 2:3:4.001' day to second +-- !query analysis +Project [INTERVAL '1 02:03:04.001' DAY TO SECOND AS INTERVAL '1 02:03:04.001' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval -'-1 2:3:4.001' day to second +-- !query analysis +Project [INTERVAL '1 02:03:04.001' DAY TO SECOND AS INTERVAL '1 02:03:04.001' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval -'1' year +-- !query analysis +Project [INTERVAL '-1' YEAR AS INTERVAL '-1' YEAR#x] ++- OneRowRelation + + +-- !query +select interval -'-1' year +-- !query analysis +Project [INTERVAL '1' YEAR AS INTERVAL '1' YEAR#x] ++- OneRowRelation + + +-- !query +select interval -'11' month +-- !query analysis +Project [INTERVAL '-11' MONTH AS INTERVAL '-11' MONTH#x] ++- OneRowRelation + + +-- !query +select interval -'-11' month +-- !query analysis +Project [INTERVAL '11' MONTH AS INTERVAL '11' MONTH#x] ++- OneRowRelation + + +-- !query +select interval -'1' day +-- !query analysis +Project [INTERVAL '-1' DAY AS INTERVAL '-1' DAY#x] ++- OneRowRelation + + +-- !query +select interval -'-1' day +-- !query analysis +Project [INTERVAL '1' DAY AS INTERVAL '1' DAY#x] ++- OneRowRelation + + +-- !query +select interval -'23' hour +-- !query analysis +Project [INTERVAL '-23' HOUR AS INTERVAL '-23' HOUR#x] ++- OneRowRelation + + +-- !query +select interval -'-23' hour +-- !query analysis +Project [INTERVAL '23' HOUR AS INTERVAL '23' HOUR#x] ++- OneRowRelation + + +-- !query +select interval -'59' minute +-- !query analysis +Project [INTERVAL '-59' MINUTE AS INTERVAL '-59' MINUTE#x] ++- OneRowRelation + + +-- !query +select interval -'-59' minute +-- !query analysis +Project [INTERVAL '59' MINUTE AS INTERVAL '59' MINUTE#x] ++- OneRowRelation + + +-- !query +select interval -'59' second +-- !query analysis +Project [INTERVAL '-59' SECOND AS INTERVAL '-59' SECOND#x] ++- OneRowRelation + + +-- !query +select interval -'-59' second +-- !query analysis +Project [INTERVAL '59' SECOND AS INTERVAL '59' SECOND#x] ++- OneRowRelation + + +-- !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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select make_dt_interval(1) +-- !query analysis +Project [make_dt_interval(1, 0, 0, 0.000000) AS make_dt_interval(1, 0, 0, 0.000000)#x] ++- OneRowRelation + + +-- !query +select make_dt_interval(1, 2) +-- !query analysis +Project [make_dt_interval(1, 2, 0, 0.000000) AS make_dt_interval(1, 2, 0, 0.000000)#x] ++- OneRowRelation + + +-- !query +select make_dt_interval(1, 2, 3) +-- !query analysis +Project [make_dt_interval(1, 2, 3, 0.000000) AS make_dt_interval(1, 2, 3, 0.000000)#x] ++- OneRowRelation + + +-- !query +select make_dt_interval(1, 2, 3, 4.005006) +-- !query analysis +Project [make_dt_interval(1, 2, 3, cast(4.005006 as decimal(18,6))) AS make_dt_interval(1, 2, 3, 4.005006)#x] ++- OneRowRelation + + +-- !query +select make_dt_interval(1, 0, 0, 123456789012.123456) +-- !query analysis +Project [make_dt_interval(1, 0, 0, 123456789012.123456) AS make_dt_interval(1, 0, 0, 123456789012.123456)#x] ++- OneRowRelation + + +-- !query +select make_dt_interval(2147483647) +-- !query analysis +Project [make_dt_interval(2147483647, 0, 0, 0.000000) AS make_dt_interval(2147483647, 0, 0, 0.000000)#x] ++- OneRowRelation + + +-- !query +select make_ym_interval(1) +-- !query analysis +Project [make_ym_interval(1, 0) AS make_ym_interval(1, 0)#x] ++- OneRowRelation + + +-- !query +select make_ym_interval(1, 2) +-- !query analysis +Project [make_ym_interval(1, 2) AS make_ym_interval(1, 2)#x] ++- OneRowRelation + + +-- !query +select make_ym_interval(0, 1) +-- !query analysis +Project [make_ym_interval(0, 1) AS make_ym_interval(0, 1)#x] ++- OneRowRelation + + +-- !query +select make_ym_interval(178956970, 7) +-- !query analysis +Project [make_ym_interval(178956970, 7) AS make_ym_interval(178956970, 7)#x] ++- OneRowRelation + + +-- !query +select make_ym_interval(178956970, 8) +-- !query analysis +Project [make_ym_interval(178956970, 8) AS make_ym_interval(178956970, 8)#x] ++- OneRowRelation + + +-- !query +select make_ym_interval(-178956970, -8) +-- !query analysis +Project [make_ym_interval(-178956970, -8) AS make_ym_interval(-178956970, -8)#x] ++- OneRowRelation + + +-- !query +select make_ym_interval(-178956970, -9) +-- !query analysis +Project [make_ym_interval(-178956970, -9) AS make_ym_interval(-178956970, -9)#x] ++- OneRowRelation + + +-- !query +select cast('1 second' as interval) +-- !query analysis +Project [cast(1 second as interval) AS CAST(1 second AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +select cast('+1 second' as interval) +-- !query analysis +Project [cast(+1 second as interval) AS CAST(+1 second AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +select cast('-1 second' as interval) +-- !query analysis +Project [cast(-1 second as interval) AS CAST(-1 second AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +select cast('+ 1 second' as interval) +-- !query analysis +Project [cast(+ 1 second as interval) AS CAST(+ 1 second AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +select cast('- 1 second' as interval) +-- !query analysis +Project [cast(- 1 second as interval) AS CAST(- 1 second AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +select cast('- -1 second' as interval) +-- !query analysis +Project [cast(- -1 second as interval) AS CAST(- -1 second AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +select cast('- +1 second' as interval) +-- !query analysis +Project [cast(- +1 second as interval) AS CAST(- +1 second AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +select interval 13.123456789 seconds, interval -13.123456789 second +-- !query analysis +Project [INTERVAL '13.123456' SECOND AS INTERVAL '13.123456' SECOND#x, INTERVAL '-13.123456' SECOND AS INTERVAL '-13.123456' SECOND#x] ++- OneRowRelation + + +-- !query +select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond" + } ] +} + + +-- !query +select interval 1 year 2 month +-- !query analysis +Project [INTERVAL '1-2' YEAR TO MONTH AS INTERVAL '1-2' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval 4 day 5 hour 6 minute 7 seconds +-- !query analysis +Project [INTERVAL '4 05:06:07' DAY TO SECOND AS INTERVAL '4 05:06:07' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval 3 week 8 millisecond 9 microsecond +-- !query analysis +Project [INTERVAL '21 00:00:00.008009' DAY TO SECOND AS INTERVAL '21 00:00:00.008009' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 92, + "fragment" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second" + } ] +} + + +-- !query +select interval '30' year '25' month +-- !query analysis +Project [INTERVAL '32-1' YEAR TO MONTH AS INTERVAL '32-1' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval '-100' day '40' hour '80' minute '299.889987299' second +-- !query analysis +Project [INTERVAL '-98 06:35:00.110013' DAY TO SECOND AS INTERVAL '-98 06:35:00.110013' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '0-0' year to month +-- !query analysis +Project [INTERVAL '0-0' YEAR TO MONTH AS INTERVAL '0-0' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval '0 0:0:0' day to second +-- !query analysis +Project [INTERVAL '0 00:00:00' DAY TO SECOND AS INTERVAL '0 00:00:00' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '0 0:0:0.1' day to second +-- !query analysis +Project [INTERVAL '0 00:00:00.1' DAY TO SECOND AS INTERVAL '0 00:00:00.1' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '10-9' year to month +-- !query analysis +Project [INTERVAL '10-9' YEAR TO MONTH AS INTERVAL '10-9' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval '20 15' day to hour +-- !query analysis +Project [INTERVAL '20 15' DAY TO HOUR AS INTERVAL '20 15' DAY TO HOUR#x] ++- OneRowRelation + + +-- !query +select interval '20 15:40' day to minute +-- !query analysis +Project [INTERVAL '20 15:40' DAY TO MINUTE AS INTERVAL '20 15:40' DAY TO MINUTE#x] ++- OneRowRelation + + +-- !query +select interval '20 15:40:32.99899999' day to second +-- !query analysis +Project [INTERVAL '20 15:40:32.998999' DAY TO SECOND AS INTERVAL '20 15:40:32.998999' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '15:40' hour to minute +-- !query analysis +Project [INTERVAL '15:40' HOUR TO MINUTE AS INTERVAL '15:40' HOUR TO MINUTE#x] ++- OneRowRelation + + +-- !query +select interval '15:40:32.99899999' hour to second +-- !query analysis +Project [INTERVAL '15:40:32.998999' HOUR TO SECOND AS INTERVAL '15:40:32.998999' HOUR TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '40:32.99899999' minute to second +-- !query analysis +Project [INTERVAL '40:32.998999' MINUTE TO SECOND AS INTERVAL '40:32.998999' MINUTE TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '40:32' minute to second +-- !query analysis +Project [INTERVAL '40:32' MINUTE TO SECOND AS INTERVAL '40:32' MINUTE TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval 30 day day +-- !query analysis +Project [INTERVAL '30' DAY AS day#x] ++- OneRowRelation + + +-- !query +select interval 30 days days +-- !query analysis +Project [INTERVAL '30' DAY AS days#x] ++- OneRowRelation + + +-- !query +select interval '20 15:40:32.99899999' day to hour +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 50, + "fragment" : "'20 15:40:32.99899999' day to hour" + } ] +} + + +-- !query +select interval '20 15:40:32.99899999' day to minute +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE` when cast to interval day to minute: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 52, + "fragment" : "'20 15:40:32.99899999' day to minute" + } ] +} + + +-- !query +select interval '15:40:32.99899999' hour to minute +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 50, + "fragment" : "'15:40:32.99899999' hour to minute" + } ] +} + + +-- !query +select interval '15:40.99899999' hour to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 47, + "fragment" : "'15:40.99899999' hour to second" + } ] +} + + +-- !query +select interval '15:40' hour to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 38, + "fragment" : "'15:40' hour to second" + } ] +} + + +-- !query +select interval '20 40:32.99899999' minute to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 20 40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 52, + "fragment" : "'20 40:32.99899999' minute to second" + } ] +} + + +-- !query +select interval 10 nanoseconds +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0062", + "messageParameters" : { + "msg" : "Error parsing ' 10 nanoseconds' to interval, invalid unit 'nanoseconds'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 30, + "fragment" : "10 nanoseconds" + } ] +} + + +-- !query +select map(1, interval 1 day, 2, interval 3 week) +-- !query analysis +Project [map(1, INTERVAL '1' DAY, 2, INTERVAL '21' DAY) AS map(1, INTERVAL '1' DAY, 2, INTERVAL '21' DAY)#x] ++- OneRowRelation + + +-- !query +select map(1, interval 1 day, 2, interval 2 day) +-- !query analysis +Project [map(1, INTERVAL '1' DAY, 2, INTERVAL '2' DAY) AS map(1, INTERVAL '1' DAY, 2, INTERVAL '2' DAY)#x] ++- OneRowRelation + + +-- !query +select map(1, interval 1 year, 2, interval 2 month) +-- !query analysis +Project [map(1, cast(INTERVAL '1' YEAR as interval year to month), 2, cast(INTERVAL '2' MONTH as interval year to month)) AS map(1, INTERVAL '1' YEAR, 2, INTERVAL '2' MONTH)#x] ++- OneRowRelation + + +-- !query +select map(1, interval 1 month, 2, interval 2 month) +-- !query analysis +Project [map(1, INTERVAL '1' MONTH, 2, INTERVAL '2' MONTH) AS map(1, INTERVAL '1' MONTH, 2, INTERVAL '2' MONTH)#x] ++- OneRowRelation + + +-- !query +select map(1, interval 1 week, 2, interval 2 day) +-- !query analysis +Project [map(1, INTERVAL '7' DAY, 2, INTERVAL '2' DAY) AS map(1, INTERVAL '7' DAY, 2, INTERVAL '2' DAY)#x] ++- OneRowRelation + + +-- !query +select map(1, interval 2 millisecond, 3, interval 3 microsecond) +-- !query analysis +Project [map(1, INTERVAL '00.002' SECOND, 3, INTERVAL '00.000003' SECOND) AS map(1, INTERVAL '00.002' SECOND, 3, INTERVAL '00.000003' SECOND)#x] ++- OneRowRelation + + +-- !query +select interval 'interval 3 year 1 month' +-- !query analysis +Project [INTERVAL '3-1' YEAR TO MONTH AS INTERVAL '3-1' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval '3 year 1 month' +-- !query analysis +Project [INTERVAL '3-1' YEAR TO MONTH AS INTERVAL '3-1' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +SELECT interval 'interval 2 weeks 2 days 1 hour 3 minutes 2 seconds 100 millisecond 200 microseconds' +-- !query analysis +Project [INTERVAL '16 01:03:02.1002' DAY TO SECOND AS INTERVAL '16 01:03:02.1002' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +SELECT interval '2 weeks 2 days 1 hour 3 minutes 2 seconds 100 millisecond 200 microseconds' +-- !query analysis +Project [INTERVAL '16 01:03:02.1002' DAY TO SECOND AS INTERVAL '16 01:03:02.1002' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval 1 fake_unit +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'fake_unit'", + "hint" : "" + } +} + + +-- !query +select interval 1 year to month +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0027", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 31, + "fragment" : "1 year to month" + } ] +} + + +-- !query +select interval '1' year to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0028", + "messageParameters" : { + "from" : "year", + "to" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 34, + "fragment" : "'1' year to second" + } ] +} + + +-- !query +select interval '10-9' year to month '2-1' year to month +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "interval '10-9' year to month '2-1' year to month" + } ] +} + + +-- !query +select interval '10-9' year to month '12:11:10' hour to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "interval '10-9' year to month '12:11:10' hour to second" + } ] +} + + +-- !query +select interval '1 15:11' day to minute '12:11:10' hour to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 65, + "fragment" : "interval '1 15:11' day to minute '12:11:10' hour to second" + } ] +} + + +-- !query +select interval 1 year '2-1' year to month +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "interval 1 year '2-1' year to month" + } ] +} + + +-- !query +select interval 1 year '12:11:10' hour to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "interval 1 year '12:11:10' hour to second" + } ] +} + + +-- !query +select interval '10-9' year to month '1' year +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "interval '10-9' year to month '1' year" + } ] +} + + +-- !query +select interval '12:11:10' hour to second '1' year +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "interval '12:11:10' hour to second '1' year" + } ] +} + + +-- !query +select interval (-30) day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval (-30)" + } ] +} + + +-- !query +select interval (a + 1) day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "interval (a + 1)" + } ] +} + + +-- !query +select interval 30 day day day +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'day'", + "hint" : ": extra input 'day'" + } +} + + +-- !query +select interval (-30) days +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval (-30)" + } ] +} + + +-- !query +select interval (a + 1) days +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "interval (a + 1)" + } ] +} + + +-- !query +select interval 30 days days days +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'days'", + "hint" : ": extra input 'days'" + } +} + + +-- !query +SELECT INTERVAL '178956970-7' YEAR TO MONTH +-- !query analysis +Project [INTERVAL '178956970-7' YEAR TO MONTH AS INTERVAL '178956970-7' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '178956970-8' YEAR TO MONTH +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Error parsing interval year-month string: integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 43, + "fragment" : "'178956970-8' YEAR TO MONTH" + } ] +} + + +-- !query +SELECT INTERVAL '-178956970-8' YEAR TO MONTH +-- !query analysis +Project [INTERVAL '-178956970-8' YEAR TO MONTH AS INTERVAL '-178956970-8' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL -'178956970-8' YEAR TO MONTH +-- !query analysis +Project [INTERVAL '-178956970-8' YEAR TO MONTH AS INTERVAL '-178956970-8' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select + interval '2-2' year to month + interval '3' month, + interval '2' year - interval '3-3' year to month, + interval '99 11:22:33.123456789' day to second + interval '10 9:8' day to minute, + interval '22:33.123456789' minute to second - interval '10' day +-- !query analysis +Project [(INTERVAL '2-2' YEAR TO MONTH + cast(INTERVAL '3' MONTH as interval year to month)) AS (INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' MONTH)#x, (cast(INTERVAL '2' YEAR as interval year to month) - INTERVAL '3-3' YEAR TO MONTH) AS (INTERVAL '2' YEAR - INTERVAL '3-3' YEAR TO MONTH)#x, (INTERVAL '99 11:22:33.123456' DAY TO SECOND + cast(INTERVAL '10 09:08' DAY TO MINUTE as interval day to second)) AS (INTERVAL '99 11:22:33.123456' DAY TO SECOND + INTERVAL '10 09:08' DAY TO MINUTE)#x, (cast(INTERVAL '22:33.123456' MINUTE TO SECOND as interval day to second) - cast(INTERVAL '10' DAY as interval day to second)) AS (INTERVAL '22:33.123456' MINUTE TO SECOND - INTERVAL '10' DAY)#x] ++- OneRowRelation + + +-- !query +select + interval '2' year + '3-3 year to month', + interval '2' year - '3 month', + '3-2 year to month' + interval '2-2' year to month, + '3 year' - interval '2-2' year to month, + interval '99 11:22:33.123456789' day to second + '12:12 hour to second', + interval '99 11:22:33.123456789' day to second - '12 hour', + '4 day' + interval '10' day, + '4 22 day to hour' - interval '10' day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3 year to month)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 48, + "fragment" : "interval '2' year + '3-3 year to month'" + } ] +} + + +-- !query +select + interval '2' year + null, + interval '2' year - null, + interval '2' hour + null, + interval '2' hour - null, + null + interval '2' year, + null - interval '2' year, + null + interval '2' hour, + null - interval '2' hour +-- !query analysis +Project [(INTERVAL '2' YEAR + cast(null as interval year)) AS (INTERVAL '2' YEAR + NULL)#x, (INTERVAL '2' YEAR - cast(null as interval year)) AS (INTERVAL '2' YEAR - NULL)#x, (INTERVAL '02' HOUR + cast(null as interval hour)) AS (INTERVAL '02' HOUR + NULL)#x, (INTERVAL '02' HOUR - cast(null as interval hour)) AS (INTERVAL '02' HOUR - NULL)#x, (cast(null as interval year) + INTERVAL '2' YEAR) AS (NULL + INTERVAL '2' YEAR)#x, (cast(null as interval year) - INTERVAL '2' YEAR) AS (NULL - INTERVAL '2' YEAR)#x, (cast(null as interval hour) + INTERVAL '02' HOUR) AS (NULL + INTERVAL '02' HOUR)#x, (cast(null as interval hour) - INTERVAL '02' HOUR) AS (NULL - INTERVAL '02' HOUR)#x] ++- OneRowRelation + + +-- !query +select interval '2' year + '3-3' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "interval '2' year + '3-3'" + } ] +} + + +-- !query +select interval '2' year - '4' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR - 4)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year - '4'" + } ] +} + + +-- !query +select '4 11:11' - interval '4 22:12' day to minute +-- !query analysis +Project [cast(4 11:11 - INTERVAL '4 22:12' DAY TO MINUTE as string) AS 4 11:11 - INTERVAL '4 22:12' DAY TO MINUTE#x] ++- OneRowRelation + + +-- !query +select '4 12:12:12' + interval '4 22:12' day to minute +-- !query analysis +Project [cast(cast(4 12:12:12 as timestamp) + INTERVAL '4 22:12' DAY TO MINUTE as string) AS 4 12:12:12 + INTERVAL '4 22:12' DAY TO MINUTE#x] ++- OneRowRelation + + +-- !query +create temporary view interval_view as select '1' str +-- !query analysis +CreateViewCommand `interval_view`, select '1' str, false, false, LocalTempView, true + +- Project [1 AS str#x] + +- OneRowRelation + + +-- !query +select interval '2' year + str from interval_view +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year + str" + } ] +} + + +-- !query +select interval '2' year - str from interval_view +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR - str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year - str" + } ] +} + + +-- !query +select str - interval '4 22:12' day to minute from interval_view +-- !query analysis +Project [cast(str#x - INTERVAL '4 22:12' DAY TO MINUTE as string) AS str - INTERVAL '4 22:12' DAY TO MINUTE#x] ++- SubqueryAlias interval_view + +- View (`interval_view`, [str#x]) + +- Project [cast(str#x as string) AS str#x] + +- Project [1 AS str#x] + +- OneRowRelation + + +-- !query +select str + interval '4 22:12' day to minute from interval_view +-- !query analysis +Project [cast(cast(str#x as timestamp) + INTERVAL '4 22:12' DAY TO MINUTE as string) AS str + INTERVAL '4 22:12' DAY TO MINUTE#x] ++- SubqueryAlias interval_view + +- View (`interval_view`, [str#x]) + +- Project [cast(str#x as string) AS str#x] + +- Project [1 AS str#x] + +- OneRowRelation + + +-- !query +select interval '2-2' year to month + interval '3' day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"", + "inputType" : "\"INTERVAL YEAR TO MONTH\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '2-2' year to month + interval '3' day" + } ] +} + + +-- !query +select interval '3' day + interval '2-2' year to month +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"", + "inputType" : "\"INTERVAL YEAR TO MONTH\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '3' day + interval '2-2' year to month" + } ] +} + + +-- !query +select interval '2-2' year to month - interval '3' day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"", + "inputType" : "\"INTERVAL YEAR TO MONTH\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + (- INTERVAL '3' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '2-2' year to month - interval '3' day" + } ] +} + + +-- !query +select interval '3' day - interval '2-2' year to month +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"INTERVAL YEAR TO MONTH\"", + "sqlExpr" : "\"(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '3' day - interval '2-2' year to month" + } ] +} + + +-- !query +select 1 - interval '2' second +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"1 + (- INTERVAL '02' SECOND)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "1 - interval '2' second" + } ] +} + + +-- !query +select 1 + interval '2' month +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"INTERVAL MONTH\"", + "sqlExpr" : "\"(1 + INTERVAL '2' MONTH)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "1 + interval '2' month" + } ] +} + + +-- !query +select interval '2' second + 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"1 + INTERVAL '02' SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' second + 1" + } ] +} + + +-- !query +select interval '2' month - 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL MONTH\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(INTERVAL '2' MONTH - 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "interval '2' month - 1" + } ] +} + + +-- !query +select interval '\t interval 1 day' +-- !query analysis +Project [INTERVAL '1' DAY AS INTERVAL '1' DAY#x] ++- OneRowRelation + + +-- !query +select interval 'interval \t 1\tday' +-- !query analysis +Project [INTERVAL '1' DAY AS INTERVAL '1' DAY#x] ++- OneRowRelation + + +-- !query +select interval 'interval\t1\tday' +-- !query analysis +Project [INTERVAL '1' DAY AS INTERVAL '1' DAY#x] ++- OneRowRelation + + +-- !query +select interval '1\t' day +-- !query analysis +Project [INTERVAL '1' DAY AS INTERVAL '1' DAY#x] ++- OneRowRelation + + +-- !query +select interval '1 ' day +-- !query analysis +Project [INTERVAL '1' DAY AS INTERVAL '1' DAY#x] ++- OneRowRelation + + +-- !query +select interval '2-2\t' year to month +-- !query analysis +Project [INTERVAL '2-2' YEAR TO MONTH AS INTERVAL '2-2' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval '-\t2-2\t' year to month +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match year-month format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval year to month: -\t2-2\t" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 40, + "fragment" : "'-\\t2-2\\t' year to month" + } ] +} + + +-- !query +select interval '\n0 12:34:46.789\t' day to second +-- !query analysis +Project [INTERVAL '0 12:34:46.789' DAY TO SECOND AS INTERVAL '0 12:34:46.789' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '\n-\t10\t 12:34:46.789\t' day to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND` when cast to interval day to second: \n-\t10\t 12:34:46.789\t, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 56, + "fragment" : "'\\n-\\t10\\t 12:34:46.789\\t' day to second" + } ] +} + + +-- !query +select interval '中文 interval 1 day' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'中文 interval 1 day'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "interval '中文 interval 1 day'" + } ] +} + + +-- !query +select interval 'interval中文 1 day' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'interval中文 1 day'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "interval 'interval中文 1 day'" + } ] +} + + +-- !query +select interval 'interval 1中文day' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'interval 1中文day'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "interval 'interval 1中文day'" + } ] +} + + +-- !query +select -(a) from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query analysis +Project [-a#x AS (- a)#x] ++- SubqueryAlias t + +- LocalRelation [a#x, b#x] + + +-- !query +select a - b from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query analysis +Project [(a#x - b#x) AS (a - b)#x] ++- SubqueryAlias t + +- LocalRelation [a#x, b#x] + + +-- !query +select b + interval '1 month' from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query analysis +Project [(b#x + INTERVAL '1' MONTH) AS (b + INTERVAL '1' MONTH)#x] ++- SubqueryAlias t + +- LocalRelation [a#x, b#x] + + +-- !query +select a * 1.1 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query analysis +Project [(a#x * 1.1) AS (a * 1.1)#x] ++- SubqueryAlias t + +- LocalRelation [a#x, b#x] + + +-- !query +select a / 0.5 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query analysis +Project [(a#x / 0.5) AS (a / 0.5)#x] ++- SubqueryAlias t + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT + from_csv('1, 1 day', 'a INT, b interval'), + from_csv('1, 1', 'a INT, b interval day'), + to_csv(from_csv('1, 1 day', 'a INT, b interval')), + to_csv(from_csv('1, 1', 'a INT, b interval day')), + to_csv(named_struct('a', interval 32 hour, 'b', interval 70 minute)), + from_csv(to_csv(named_struct('a', interval 32 hour, 'b', interval 70 minute)), 'a interval hour, b interval minute') +-- !query analysis +Project [from_csv(StructField(a,IntegerType,true), StructField(b,CalendarIntervalType,true), 1, 1 day, Some(America/Los_Angeles), None) AS from_csv(1, 1 day)#x, from_csv(StructField(a,IntegerType,true), StructField(b,DayTimeIntervalType(0,0),true), 1, 1, Some(America/Los_Angeles), None) AS from_csv(1, 1)#x, to_csv(from_csv(StructField(a,IntegerType,true), StructField(b,CalendarIntervalType,true), 1, 1 day, Some(America/Los_Angeles), None), Some(America/Los_Angeles)) AS to_csv(from_csv(1, 1 day))#x, to_csv(from_csv(StructField(a,IntegerType,true), StructField(b,DayTimeIntervalType(0,0),true), 1, 1, Some(America/Los_Angeles), None), Some(America/Los_Angeles)) AS to_csv(from_csv(1, 1))#x, to_csv(named_struct(a, INTERVAL '32' HOUR, b, INTERVAL '70' MINUTE), Some(America/Los_Angeles)) AS to_csv(named_struct(a, INTERVAL '32' HOUR, b, INTERVAL '70' MINUTE))#x, from_csv(StructField(a,DayTimeIntervalType(1,1),true), StructField(b,DayTimeIntervalType(2,2),true), to_csv(named_struct(a, INTERVAL '32' HOUR, b, INTERVAL '70' MINUTE), Some(America/Los_Angeles)), Some(America/Los_Angeles), None) AS from_csv(to_csv(named_struct(a, INTERVAL '32' HOUR, b, INTERVAL '70' MINUTE)))#x] ++- OneRowRelation + + +-- !query +SELECT + from_json('{"a":"1 days"}', 'a interval'), + from_csv('1, 1', 'a INT, b interval year'), + to_json(from_json('{"a":"1 days"}', 'a interval')), + to_csv(from_csv('1, 1', 'a INT, b interval year')), + to_csv(named_struct('a', interval 32 year, 'b', interval 10 month)), + from_csv(to_csv(named_struct('a', interval 32 year, 'b', interval 10 month)), 'a interval year, b interval month') +-- !query analysis +Project [from_json(StructField(a,CalendarIntervalType,true), {"a":"1 days"}, Some(America/Los_Angeles)) AS from_json({"a":"1 days"})#x, from_csv(StructField(a,IntegerType,true), StructField(b,YearMonthIntervalType(0,0),true), 1, 1, Some(America/Los_Angeles), None) AS from_csv(1, 1)#x, to_json(from_json(StructField(a,CalendarIntervalType,true), {"a":"1 days"}, Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS to_json(from_json({"a":"1 days"}))#x, to_csv(from_csv(StructField(a,IntegerType,true), StructField(b,YearMonthIntervalType(0,0),true), 1, 1, Some(America/Los_Angeles), None), Some(America/Los_Angeles)) AS to_csv(from_csv(1, 1))#x, to_csv(named_struct(a, INTERVAL '32' YEAR, b, INTERVAL '10' MONTH), Some(America/Los_Angeles)) AS to_csv(named_struct(a, INTERVAL '32' YEAR, b, INTERVAL '10' MONTH))#x, from_csv(StructField(a,YearMonthIntervalType(0,0),true), StructField(b,YearMonthIntervalType(1,1),true), to_csv(named_struct(a, INTERVAL '32' YEAR, b, INTERVAL '10' MONTH), Some(America/Los_Angeles)), Some(America/Los_Angeles), None) AS from_csv(to_csv(named_struct(a, INTERVAL '32' YEAR, b, INTERVAL '10' MONTH)))#x] ++- OneRowRelation + + +-- !query +SELECT + from_json('{"a":"1"}', 'a interval day'), + to_json(from_json('{"a":"1"}', 'a interval day')), + to_json(map('a', interval 100 day 130 minute)), + from_json(to_json(map('a', interval 100 day 130 minute)), 'a interval day to minute') +-- !query analysis +Project [from_json(StructField(a,DayTimeIntervalType(0,0),true), {"a":"1"}, Some(America/Los_Angeles)) AS from_json({"a":"1"})#x, to_json(from_json(StructField(a,DayTimeIntervalType(0,0),true), {"a":"1"}, Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS to_json(from_json({"a":"1"}))#x, to_json(map(a, INTERVAL '100 02:10' DAY TO MINUTE), Some(America/Los_Angeles)) AS to_json(map(a, INTERVAL '100 02:10' DAY TO MINUTE))#x, from_json(StructField(a,DayTimeIntervalType(0,2),true), to_json(map(a, INTERVAL '100 02:10' DAY TO MINUTE), Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS from_json(to_json(map(a, INTERVAL '100 02:10' DAY TO MINUTE)))#x] ++- OneRowRelation + + +-- !query +SELECT + from_json('{"a":"1"}', 'a interval year'), + to_json(from_json('{"a":"1"}', 'a interval year')), + to_json(map('a', interval 32 year 10 month)), + from_json(to_json(map('a', interval 32 year 10 month)), 'a interval year to month') +-- !query analysis +Project [from_json(StructField(a,YearMonthIntervalType(0,0),true), {"a":"1"}, Some(America/Los_Angeles)) AS from_json({"a":"1"})#x, to_json(from_json(StructField(a,YearMonthIntervalType(0,0),true), {"a":"1"}, Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS to_json(from_json({"a":"1"}))#x, to_json(map(a, INTERVAL '32-10' YEAR TO MONTH), Some(America/Los_Angeles)) AS to_json(map(a, INTERVAL '32-10' YEAR TO MONTH))#x, from_json(StructField(a,YearMonthIntervalType(0,1),true), to_json(map(a, INTERVAL '32-10' YEAR TO MONTH), Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS from_json(to_json(map(a, INTERVAL '32-10' YEAR TO MONTH)))#x] ++- OneRowRelation + + +-- !query +select interval '+' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'+'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "interval '+'" + } ] +} + + +-- !query +select interval '+.' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'+.'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "interval '+.'" + } ] +} + + +-- !query +select interval '1' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "interval '1'" + } ] +} + + +-- !query +select interval '1.2' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1.2'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval '1.2'" + } ] +} + + +-- !query +select interval '- 2' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'- 2'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval '- 2'" + } ] +} + + +-- !query +select interval '1 day -' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1 day -'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "interval '1 day -'" + } ] +} + + +-- !query +select interval '1 day 1' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1 day 1'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "interval '1 day 1'" + } ] +} + + +-- !query +select interval '1 day 2' day +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0026", + "messageParameters" : { + "value" : "1 day 2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 29, + "fragment" : "'1 day 2' day" + } ] +} + + +-- !query +select interval 'interval 1' day +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0026", + "messageParameters" : { + "value" : "interval 1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 32, + "fragment" : "'interval 1' day" + } ] +} + + +-- !query +select interval '-\t 1' day +-- !query analysis +Project [INTERVAL '-1' DAY AS INTERVAL '-1' DAY#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / 2 +-- !query analysis +Project [(INTERVAL '-178956970-8' YEAR TO MONTH / 2) AS (INTERVAL '-178956970-8' YEAR TO MONTH / 2)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / 5 +-- !query analysis +Project [(INTERVAL '-178956970-8' YEAR TO MONTH / 5) AS (INTERVAL '-178956970-8' YEAR TO MONTH / 5)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1 +-- !query analysis +Project [(INTERVAL '-178956970-8' YEAR TO MONTH / -1) AS (INTERVAL '-178956970-8' YEAR TO MONTH / -1)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1L +-- !query analysis +Project [(INTERVAL '-178956970-8' YEAR TO MONTH / -1) AS (INTERVAL '-178956970-8' YEAR TO MONTH / -1)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1.0 +-- !query analysis +Project [(INTERVAL '-178956970-8' YEAR TO MONTH / -1.0) AS (INTERVAL '-178956970-8' YEAR TO MONTH / -1.0)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1.0D +-- !query analysis +Project [(INTERVAL '-178956970-8' YEAR TO MONTH / -1.0) AS (INTERVAL '-178956970-8' YEAR TO MONTH / -1.0)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / 2 +-- !query analysis +Project [(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / 2) AS (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / 2)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / 5 +-- !query analysis +Project [(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / 5) AS (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / 5)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1 +-- !query analysis +Project [(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1) AS (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L +-- !query analysis +Project [(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1) AS (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1.0 +-- !query analysis +Project [(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1.0) AS (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1.0)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1.0D +-- !query analysis +Project [(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1.0) AS (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1.0)#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '106751991 04' DAY TO HOUR +-- !query analysis +Project [INTERVAL '106751991 04' DAY TO HOUR AS INTERVAL '106751991 04' DAY TO HOUR#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '106751991 04:00' DAY TO MINUTE +-- !query analysis +Project [INTERVAL '106751991 04:00' DAY TO MINUTE AS INTERVAL '106751991 04:00' DAY TO MINUTE#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '106751991 04:00:54.775807' DAY TO SECOND +-- !query analysis +Project [INTERVAL '106751991 04:00:54.775807' DAY TO SECOND AS INTERVAL '106751991 04:00:54.775807' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '2562047788:00' HOUR TO MINUTE +-- !query analysis +Project [INTERVAL '2562047788:00' HOUR TO MINUTE AS INTERVAL '2562047788:00' HOUR TO MINUTE#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '2562047788:00:54.775807' HOUR TO SECOND +-- !query analysis +Project [INTERVAL '2562047788:00:54.775807' HOUR TO SECOND AS INTERVAL '2562047788:00:54.775807' HOUR TO SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '153722867280:54.775807' MINUTE TO SECOND +-- !query analysis +Project [INTERVAL '153722867280:54.775807' MINUTE TO SECOND AS INTERVAL '153722867280:54.775807' MINUTE TO SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-106751991 04' DAY TO HOUR +-- !query analysis +Project [INTERVAL '-106751991 04' DAY TO HOUR AS INTERVAL '-106751991 04' DAY TO HOUR#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-106751991 04:00' DAY TO MINUTE +-- !query analysis +Project [INTERVAL '-106751991 04:00' DAY TO MINUTE AS INTERVAL '-106751991 04:00' DAY TO MINUTE#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND +-- !query analysis +Project [INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND AS INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-2562047788:00' HOUR TO MINUTE +-- !query analysis +Project [INTERVAL '-2562047788:00' HOUR TO MINUTE AS INTERVAL '-2562047788:00' HOUR TO MINUTE#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-2562047788:00:54.775808' HOUR TO SECOND +-- !query analysis +Project [INTERVAL '-2562047788:00:54.775808' HOUR TO SECOND AS INTERVAL '-2562047788:00:54.775808' HOUR TO SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-153722867280:54.775808' MINUTE TO SECOND +-- !query analysis +Project [INTERVAL '-153722867280:54.775808' MINUTE TO SECOND AS INTERVAL '-153722867280:54.775808' MINUTE TO SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '106751992 04' DAY TO HOUR +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: day 106751992 outside range [0, 106751991]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 42, + "fragment" : "'106751992 04' DAY TO HOUR" + } ] +} + + +-- !query +SELECT INTERVAL '-106751992 04' DAY TO HOUR +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: day 106751992 outside range [0, 106751991]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 43, + "fragment" : "'-106751992 04' DAY TO HOUR" + } ] +} + + +-- !query +SELECT INTERVAL '2562047789:00' HOUR TO MINUTE +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 46, + "fragment" : "'2562047789:00' HOUR TO MINUTE" + } ] +} + + +-- !query +SELECT INTERVAL '-2562047789:00' HOUR TO MINUTE +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 47, + "fragment" : "'-2562047789:00' HOUR TO MINUTE" + } ] +} + + +-- !query +SELECT INTERVAL '153722867281:54.775808' MINUTE TO SECOND +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 57, + "fragment" : "'153722867281:54.775808' MINUTE TO SECOND" + } ] +} + + +-- !query +SELECT INTERVAL '-153722867281:54.775808' MINUTE TO SECOND +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 58, + "fragment" : "'-153722867281:54.775808' MINUTE TO SECOND" + } ] +} + + +-- !query +SELECT INTERVAL '178956970' YEAR +-- !query analysis +Project [INTERVAL '178956970' YEAR AS INTERVAL '178956970' YEAR#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-178956970' YEAR +-- !query analysis +Project [INTERVAL '-178956970' YEAR AS INTERVAL '-178956970' YEAR#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '2147483647' MONTH +-- !query analysis +Project [INTERVAL '2147483647' MONTH AS INTERVAL '2147483647' MONTH#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-2147483647' MONTH +-- !query analysis +Project [INTERVAL '-2147483647' MONTH AS INTERVAL '-2147483647' MONTH#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '106751991' DAY +-- !query analysis +Project [INTERVAL '106751991' DAY AS INTERVAL '106751991' DAY#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-106751991' DAY +-- !query analysis +Project [INTERVAL '-106751991' DAY AS INTERVAL '-106751991' DAY#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '2562047788' HOUR +-- !query analysis +Project [INTERVAL '2562047788' HOUR AS INTERVAL '2562047788' HOUR#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-2562047788' HOUR +-- !query analysis +Project [INTERVAL '-2562047788' HOUR AS INTERVAL '-2562047788' HOUR#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '153722867280' MINUTE +-- !query analysis +Project [INTERVAL '153722867280' MINUTE AS INTERVAL '153722867280' MINUTE#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-153722867280' MINUTE +-- !query analysis +Project [INTERVAL '-153722867280' MINUTE AS INTERVAL '-153722867280' MINUTE#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '54.775807' SECOND +-- !query analysis +Project [INTERVAL '54.775807' SECOND AS INTERVAL '54.775807' SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-54.775807' SECOND +-- !query analysis +Project [INTERVAL '-54.775807' SECOND AS INTERVAL '-54.775807' SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '1' DAY > INTERVAL '1' HOUR +-- !query analysis +Project [(cast(INTERVAL '1' DAY as interval day to hour) > cast(INTERVAL '01' HOUR as interval day to hour)) AS (INTERVAL '1' DAY > INTERVAL '01' HOUR)#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '1 02' DAY TO HOUR = INTERVAL '02:10:55' HOUR TO SECOND +-- !query analysis +Project [(cast(INTERVAL '1 02' DAY TO HOUR as interval day to second) = cast(INTERVAL '02:10:55' HOUR TO SECOND as interval day to second)) AS (INTERVAL '1 02' DAY TO HOUR = INTERVAL '02:10:55' HOUR TO SECOND)#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '1' YEAR < INTERVAL '1' MONTH +-- !query analysis +Project [(cast(INTERVAL '1' YEAR as interval year to month) < cast(INTERVAL '1' MONTH as interval year to month)) AS (INTERVAL '1' YEAR < INTERVAL '1' MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-1-1' YEAR TO MONTH = INTERVAL '-13' MONTH +-- !query analysis +Project [(INTERVAL '-1-1' YEAR TO MONTH = cast(INTERVAL '-13' MONTH as interval year to month)) AS (INTERVAL '-1-1' YEAR TO MONTH = INTERVAL '-13' MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL MONTH\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(INTERVAL '1' MONTH > INTERVAL '20' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "INTERVAL 1 MONTH > INTERVAL 20 DAYS" + } ] +} + + +-- !query +SELECT INTERVAL '1' DAY < '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY < 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "INTERVAL '1' DAY < '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' DAY = '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY = 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "INTERVAL '1' DAY = '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' DAY > '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY > 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "INTERVAL '1' DAY > '1'" + } ] +} + + +-- !query +SELECT '1' < INTERVAL '1' DAY +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 < INTERVAL '1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' < INTERVAL '1' DAY" + } ] +} + + +-- !query +SELECT '1' = INTERVAL '1' DAY +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 = INTERVAL '1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' = INTERVAL '1' DAY" + } ] +} + + +-- !query +SELECT '1' > INTERVAL '1' DAY +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 > INTERVAL '1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' > INTERVAL '1' DAY" + } ] +} + + +-- !query +SELECT INTERVAL '1' YEAR < '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR < 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "INTERVAL '1' YEAR < '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' YEAR = '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR = 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "INTERVAL '1' YEAR = '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' YEAR > '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR > 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "INTERVAL '1' YEAR > '1'" + } ] +} + + +-- !query +SELECT '1' < INTERVAL '1' YEAR +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 < INTERVAL '1' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'1' < INTERVAL '1' YEAR" + } ] +} + + +-- !query +SELECT '1' = INTERVAL '1' YEAR +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 = INTERVAL '1' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'1' = INTERVAL '1' YEAR" + } ] +} + + +-- !query +SELECT '1' > INTERVAL '1' YEAR +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 > INTERVAL '1' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'1' > INTERVAL '1' YEAR" + } ] +} + + +-- !query +SELECT array(INTERVAL '1' YEAR, INTERVAL '1' MONTH) +-- !query analysis +Project [array(cast(INTERVAL '1' YEAR as interval year to month), cast(INTERVAL '1' MONTH as interval year to month)) AS array(INTERVAL '1' YEAR, INTERVAL '1' MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT array(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE) +-- !query analysis +Project [array(cast(INTERVAL '1' DAY as interval day to minute), cast(INTERVAL '01:01' HOUR TO MINUTE as interval day to minute)) AS array(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE)#x] ++- OneRowRelation + + +-- !query +SELECT array(INTERVAL 1 MONTH, INTERVAL 20 DAYS) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"INTERVAL MONTH\" or \"INTERVAL DAY\")", + "functionName" : "`array`", + "sqlExpr" : "\"array(INTERVAL '1' MONTH, INTERVAL '20' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "array(INTERVAL 1 MONTH, INTERVAL 20 DAYS)" + } ] +} + + +-- !query +SELECT coalesce(INTERVAL '1' YEAR, INTERVAL '1' MONTH) +-- !query analysis +Project [coalesce(cast(INTERVAL '1' YEAR as interval year to month), cast(INTERVAL '1' MONTH as interval year to month)) AS coalesce(INTERVAL '1' YEAR, INTERVAL '1' MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT coalesce(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE) +-- !query analysis +Project [coalesce(cast(INTERVAL '1' DAY as interval day to minute), cast(INTERVAL '01:01' HOUR TO MINUTE as interval day to minute)) AS coalesce(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE)#x] ++- OneRowRelation + + +-- !query +SELECT coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"INTERVAL MONTH\" or \"INTERVAL DAY\")", + "functionName" : "`coalesce`", + "sqlExpr" : "\"coalesce(INTERVAL '1' MONTH, INTERVAL '20' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS)" + } ] +} + + +-- !query +SELECT abs(INTERVAL '-10' YEAR) +-- !query analysis +Project [abs(INTERVAL '-10' YEAR) AS abs(INTERVAL '-10' YEAR)#x] ++- OneRowRelation + + +-- !query +SELECT abs(INTERVAL -'1 02:03:04.123' DAY TO SECOND) +-- !query analysis +Project [abs(INTERVAL '-1 02:03:04.123' DAY TO SECOND) AS abs(INTERVAL '-1 02:03:04.123' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +SELECT div(INTERVAL '1-1' YEAR TO MONTH, INTERVAL '1' YEAR) +-- !query analysis +Project [(INTERVAL '1-1' YEAR TO MONTH div cast(INTERVAL '1' YEAR as interval year to month)) AS (INTERVAL '1-1' YEAR TO MONTH div INTERVAL '1' YEAR)#xL] ++- OneRowRelation + + +-- !query +SELECT div(INTERVAL '1-1' YEAR TO MONTH, INTERVAL '-1' MONTH) +-- !query analysis +Project [(INTERVAL '1-1' YEAR TO MONTH div cast(INTERVAL '-1' MONTH as interval year to month)) AS (INTERVAL '1-1' YEAR TO MONTH div INTERVAL '-1' MONTH)#xL] ++- OneRowRelation + + +-- !query +SELECT div(INTERVAL '1 06' DAY TO HOUR, INTERVAL '1' DAY) +-- !query analysis +Project [(INTERVAL '1 06' DAY TO HOUR div cast(INTERVAL '1' DAY as interval day to hour)) AS (INTERVAL '1 06' DAY TO HOUR div INTERVAL '1' DAY)#xL] ++- OneRowRelation + + +-- !query +SELECT div(INTERVAL '1 06' DAY TO HOUR, INTERVAL '-1' HOUR) +-- !query analysis +Project [(INTERVAL '1 06' DAY TO HOUR div cast(INTERVAL '-01' HOUR as interval day to hour)) AS (INTERVAL '1 06' DAY TO HOUR div INTERVAL '-01' HOUR)#xL] ++- OneRowRelation + + +-- !query +SELECT div(INTERVAL '1' MONTH, INTERVAL '-1' DAY) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL MONTH\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(INTERVAL '1' MONTH div INTERVAL '-1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "div(INTERVAL '1' MONTH, INTERVAL '-1' DAY)" + } ] +} + + +-- !query +SELECT signum(INTERVAL '-10' YEAR) +-- !query analysis +Project [SIGNUM(INTERVAL '-10' YEAR) AS SIGNUM(INTERVAL '-10' YEAR)#x] ++- OneRowRelation + + +-- !query +SELECT signum(INTERVAL '10' MONTH) +-- !query analysis +Project [SIGNUM(INTERVAL '10' MONTH) AS SIGNUM(INTERVAL '10' MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT signum(INTERVAL '0-0' YEAR TO MONTH) +-- !query analysis +Project [SIGNUM(INTERVAL '0-0' YEAR TO MONTH) AS SIGNUM(INTERVAL '0-0' YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT signum(INTERVAL '-10' DAY) +-- !query analysis +Project [SIGNUM(INTERVAL '-10' DAY) AS SIGNUM(INTERVAL '-10' DAY)#x] ++- OneRowRelation + + +-- !query +SELECT signum(INTERVAL '10' HOUR) +-- !query analysis +Project [SIGNUM(INTERVAL '10' HOUR) AS SIGNUM(INTERVAL '10' HOUR)#x] ++- OneRowRelation + + +-- !query +SELECT signum(INTERVAL '0 0:0:0' DAY TO SECOND) +-- !query analysis +Project [SIGNUM(INTERVAL '0 00:00:00' DAY TO SECOND) AS SIGNUM(INTERVAL '0 00:00:00' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +SELECT width_bucket(INTERVAL '0' YEAR, INTERVAL '0' YEAR, INTERVAL '10' YEAR, 10) +-- !query analysis +Project [width_bucket(INTERVAL '0' YEAR, INTERVAL '0' YEAR, INTERVAL '10' YEAR, cast(10 as bigint)) AS width_bucket(INTERVAL '0' YEAR, INTERVAL '0' YEAR, INTERVAL '10' YEAR, 10)#xL] ++- OneRowRelation + + +-- !query +SELECT width_bucket(INTERVAL '-1' YEAR, INTERVAL -'1-2' YEAR TO MONTH, INTERVAL '1-2' YEAR TO MONTH, 10) +-- !query analysis +Project [width_bucket(INTERVAL '-1' YEAR, INTERVAL '-1-2' YEAR TO MONTH, INTERVAL '1-2' YEAR TO MONTH, cast(10 as bigint)) AS width_bucket(INTERVAL '-1' YEAR, INTERVAL '-1-2' YEAR TO MONTH, INTERVAL '1-2' YEAR TO MONTH, 10)#xL] ++- OneRowRelation + + +-- !query +SELECT width_bucket(INTERVAL '0' DAY, INTERVAL '0' DAY, INTERVAL '10' DAY, 10) +-- !query analysis +Project [width_bucket(INTERVAL '0' DAY, INTERVAL '0' DAY, INTERVAL '10' DAY, cast(10 as bigint)) AS width_bucket(INTERVAL '0' DAY, INTERVAL '0' DAY, INTERVAL '10' DAY, 10)#xL] ++- OneRowRelation + + +-- !query +SELECT width_bucket(INTERVAL '-59' MINUTE, INTERVAL -'1 01' DAY TO HOUR, INTERVAL '1 2:3:4.001' DAY TO SECOND, 10) +-- !query analysis +Project [width_bucket(INTERVAL '-59' MINUTE, INTERVAL '-1 01' DAY TO HOUR, INTERVAL '1 02:03:04.001' DAY TO SECOND, cast(10 as bigint)) AS width_bucket(INTERVAL '-59' MINUTE, INTERVAL '-1 01' DAY TO HOUR, INTERVAL '1 02:03:04.001' DAY TO SECOND, 10)#xL] ++- OneRowRelation 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/ansi/literals.sql.out new file mode 100644 index 0000000000000..9e908ad798e41 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/literals.sql.out @@ -0,0 +1,694 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select null, Null, nUll +-- !query analysis +Project [null AS NULL#x, null AS NULL#x, null AS NULL#x] ++- OneRowRelation + + +-- !query +select true, tRue, false, fALse +-- !query analysis +Project [true AS true#x, true AS true#x, false AS false#x, false AS false#x] ++- OneRowRelation + + +-- !query +select 1Y +-- !query analysis +Project [1 AS 1#x] ++- OneRowRelation + + +-- !query +select 127Y, -128Y +-- !query analysis +Project [127 AS 127#x, -128 AS -128#x] ++- OneRowRelation + + +-- !query +select 128Y +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "127", + "minValue" : "-128", + "rawStrippedQualifier" : "128", + "typeName" : "tinyint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 11, + "fragment" : "128Y" + } ] +} + + +-- !query +select 1S +-- !query analysis +Project [1 AS 1#x] ++- OneRowRelation + + +-- !query +select 32767S, -32768S +-- !query analysis +Project [32767 AS 32767#x, -32768 AS -32768#x] ++- OneRowRelation + + +-- !query +select 32768S +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "32767", + "minValue" : "-32768", + "rawStrippedQualifier" : "32768", + "typeName" : "smallint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : "32768S" + } ] +} + + +-- !query +select 1L, 2147483648L +-- !query analysis +Project [1 AS 1#xL, 2147483648 AS 2147483648#xL] ++- OneRowRelation + + +-- !query +select 9223372036854775807L, -9223372036854775808L +-- !query analysis +Project [9223372036854775807 AS 9223372036854775807#xL, -9223372036854775808 AS -9223372036854775808#xL] ++- OneRowRelation + + +-- !query +select 9223372036854775808L +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "9223372036854775807", + "minValue" : "-9223372036854775808", + "rawStrippedQualifier" : "9223372036854775808", + "typeName" : "bigint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "9223372036854775808L" + } ] +} + + +-- !query +select 1, -1 +-- !query analysis +Project [1 AS 1#x, -1 AS -1#x] ++- OneRowRelation + + +-- !query +select 2147483647, -2147483648 +-- !query analysis +Project [2147483647 AS 2147483647#x, -2147483648 AS -2147483648#x] ++- OneRowRelation + + +-- !query +select 9223372036854775807, -9223372036854775808 +-- !query analysis +Project [9223372036854775807 AS 9223372036854775807#xL, -9223372036854775808 AS -9223372036854775808#xL] ++- OneRowRelation + + +-- !query +select 9223372036854775808, -9223372036854775809 +-- !query analysis +Project [9223372036854775808 AS 9223372036854775808#x, -9223372036854775809 AS -9223372036854775809#x] ++- OneRowRelation + + +-- !query +select 1234567890123456789012345678901234567890 +-- !query analysis +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", + "sqlState" : "22003", + "messageParameters" : { + "maxPrecision" : "38", + "precision" : "40" + } +} + + +-- !query +select 1234567890123456789012345678901234567890.0 +-- !query analysis +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", + "sqlState" : "22003", + "messageParameters" : { + "maxPrecision" : "38", + "precision" : "41" + } +} + + +-- !query +select 1F, 1.2F, .10f, 0.10f +-- !query analysis +Project [1.0 AS 1.0#x, 1.2 AS 1.2#x, 0.1 AS 0.1#x, 0.1 AS 0.1#x] ++- OneRowRelation + + +-- !query +select -1F, -1.2F, -.10F, -0.10F +-- !query analysis +Project [-1.0 AS -1.0#x, -1.2 AS -1.2#x, -0.1 AS -0.1#x, -0.1 AS -0.1#x] ++- OneRowRelation + + +-- !query +select -3.4028235E39f +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "3.4028234663852886E+38", + "minValue" : "-3.4028234663852886E+38", + "rawStrippedQualifier" : "-3.4028235E39", + "typeName" : "float" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "-3.4028235E39f" + } ] +} + + +-- !query +select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 +-- !query analysis +Project [1.0 AS 1.0#x, 1.2 AS 1.2#x, 1.0E10 AS 1.0E10#x, 150000.0 AS 150000.0#x, 0.1 AS 0.1#x, 0.1 AS 0.1#x, 10000.0 AS 10000.0#x, 90.0 AS 90.0#x, 90.0 AS 90.0#x, 90.0 AS 90.0#x, 90.0 AS 90.0#x] ++- OneRowRelation + + +-- !query +select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5 +-- !query analysis +Project [-1.0 AS -1.0#x, -1.2 AS -1.2#x, -1.0E10 AS -1.0E10#x, -150000.0 AS -150000.0#x, -0.1 AS -0.1#x, -0.1 AS -0.1#x, -10000.0 AS -10000.0#x] ++- OneRowRelation + + +-- !query +select .e3 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'.'", + "hint" : "" + } +} + + +-- !query +select 1E309, -1E309 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "1.7976931348623157E+308", + "minValue" : "-1.7976931348623157E+308", + "rawStrippedQualifier" : "1E309", + "typeName" : "double" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "1E309" + } ] +} + + +-- !query +select 0.3, -0.8, .5, -.18, 0.1111, .1111 +-- !query analysis +Project [0.3 AS 0.3#x, -0.8 AS -0.8#x, 0.5 AS 0.5#x, -0.18 AS -0.18#x, 0.1111 AS 0.1111#x, 0.1111 AS 0.1111#x] ++- OneRowRelation + + +-- !query +select 0.3 F, 0.4 D, 0.5 BD +-- !query analysis +Project [0.3 AS F#x, 0.4 AS D#x, 0.5 AS BD#x] ++- OneRowRelation + + +-- !query +select 123456789012345678901234567890123456789e10d, 123456789012345678901234567890123456789.1e10d +-- !query analysis +Project [1.2345678901234568E48 AS 1.2345678901234568E48#x, 1.2345678901234568E48 AS 1.2345678901234568E48#x] ++- OneRowRelation + + +-- !query +select "Hello Peter!", 'hello lee!' +-- !query analysis +Project [Hello Peter! AS Hello Peter!#x, hello lee! AS hello lee!#x] ++- OneRowRelation + + +-- !query +select 'hello' 'world', 'hello' " " 'lee' +-- !query analysis +Project [helloworld AS helloworld#x, hello lee AS hello lee#x] ++- OneRowRelation + + +-- !query +select "hello 'peter'" +-- !query analysis +Project [hello 'peter' AS hello 'peter'#x] ++- OneRowRelation + + +-- !query +select 'pattern%', 'no-pattern\%', 'pattern\\%', 'pattern\\\%' +-- !query analysis +Project [pattern% AS pattern%#x, no-pattern\% AS no-pattern\%#x, pattern\% AS pattern\%#x, pattern\\% AS pattern\\%#x] ++- OneRowRelation + + +-- !query +select '\'', '"', '\n', '\r', '\t', 'Z' +-- !query analysis +Project [' AS '#x, " AS "#x, + AS +#x, AS #x, AS #x, Z AS Z#x] ++- OneRowRelation + + +-- !query +select '\110\145\154\154\157\041' +-- !query analysis +Project [Hello! AS Hello!#x] ++- OneRowRelation + + +-- !query +select '\u0057\u006F\u0072\u006C\u0064\u0020\u003A\u0029' +-- !query analysis +Project [World :) AS World :)#x] ++- OneRowRelation + + +-- !query +select dAte '2016-03-12' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date 'mar 11 2016' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'mar 11 2016'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date 'mar 11 2016'" + } ] +} + + +-- !query +select tImEstAmp '2016-03-11 20:54:00.000' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp '2016-33-11 20:54:00.000' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2016-33-11 20:54:00.000'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "timestamp '2016-33-11 20:54:00.000'" + } ] +} + + +-- !query +select GEO '(10,-6)' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_TYPED_LITERAL", + "sqlState" : "0A000", + "messageParameters" : { + "supportedTypes" : "\"DATE\", \"TIMESTAMP_NTZ\", \"TIMESTAMP_LTZ\", \"TIMESTAMP\", \"INTERVAL\", \"X\"", + "unsupportedType" : "\"GEO\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "GEO '(10,-6)'" + } ] +} + + +-- !query +select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD +-- !query analysis +Project [90912830918230182310293801923652346786 AS 90912830918230182310293801923652346786#x, 1.230E-26 AS 1.230E-26#x, 123.08 AS 123.08#x] ++- OneRowRelation + + +-- !query +select 1.20E-38BD +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0061", + "messageParameters" : { + "msg" : "[DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION] Decimal precision 40 exceeds max precision 38." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "1.20E-38BD" + } ] +} + + +-- !query +select x'2379ACFe' +-- !query analysis +Project [0x2379ACFE AS X'2379ACFE'#x] ++- OneRowRelation + + +-- !query +select X'XuZ' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'XuZ'", + "valueType" : "\"X\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : "X'XuZ'" + } ] +} + + +-- !query +SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 +-- !query analysis +Project [3.14 AS 3.14#x, -3.14 AS -3.14#x, 3.14E8 AS 3.14E8#x, 3.14E-8 AS 3.14E-8#x, -3.14E8 AS -3.14E8#x, -3.14E-8 AS -3.14E-8#x, 3.14E8 AS 3.14E8#x, 3.14E8 AS 3.14E8#x, 3.14E-8 AS 3.14E-8#x] ++- OneRowRelation + + +-- !query +select +date '1999-01-01' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"DATE '1999-01-01'\"", + "inputType" : "\"DATE\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ DATE '1999-01-01')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "+date '1999-01-01'" + } ] +} + + +-- !query +select +timestamp '1999-01-01' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"TIMESTAMP '1999-01-01 00:00:00'\"", + "inputType" : "\"TIMESTAMP\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ TIMESTAMP '1999-01-01 00:00:00')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "+timestamp '1999-01-01'" + } ] +} + + +-- !query +select +interval '1 day' +-- !query analysis +Project [positive(INTERVAL '1' DAY) AS (+ INTERVAL '1' DAY)#x] ++- OneRowRelation + + +-- !query +select +map(1, 2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"map(1, 2)\"", + "inputType" : "\"MAP\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ map(1, 2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "+map(1, 2)" + } ] +} + + +-- !query +select +array(1,2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"array(1, 2)\"", + "inputType" : "\"ARRAY\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ array(1, 2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "+array(1,2)" + } ] +} + + +-- !query +select +named_struct('a', 1, 'b', 'spark') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"named_struct(a, 1, b, spark)\"", + "inputType" : "\"STRUCT\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ named_struct(a, 1, b, spark))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "+named_struct('a', 1, 'b', 'spark')" + } ] +} + + +-- !query +select +X'1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"X'01'\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ X'01')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "+X'1'" + } ] +} + + +-- !query +select -date '1999-01-01' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"DATE '1999-01-01'\"", + "inputType" : "\"DATE\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(- DATE '1999-01-01')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "-date '1999-01-01'" + } ] +} + + +-- !query +select -timestamp '1999-01-01' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"TIMESTAMP '1999-01-01 00:00:00'\"", + "inputType" : "\"TIMESTAMP\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(- TIMESTAMP '1999-01-01 00:00:00')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "-timestamp '1999-01-01'" + } ] +} + + +-- !query +select -x'2379ACFe' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"X'2379ACFE'\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(- X'2379ACFE')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "-x'2379ACFe'" + } ] +} 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/ansi/map.sql.out new file mode 100644 index 0000000000000..c55250e4dcb9b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/map.sql.out @@ -0,0 +1,103 @@ +-- Automatically generated by SQLQueryTestSuite +-- !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] ++- OneRowRelation + + +-- !query +select map(1, 'a', 2, 'b')[5] +-- !query analysis +Project [map(1, a, 2, b)[5] AS map(1, a, 2, b)[5]#x] ++- OneRowRelation + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 5) +-- !query analysis +Project [map_contains_key(map(1, a, 2, b), 5) AS map_contains_key(map(1, a, 2, b), 5)#x] ++- OneRowRelation + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 1) +-- !query analysis +Project [map_contains_key(map(1, a, 2, b), 1) AS map_contains_key(map(1, a, 2, b), 1)#x] ++- OneRowRelation + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 5.0) +-- !query analysis +Project [map_contains_key(cast(map(1, a, 2, b) as map), cast(5.0 as decimal(11,1))) AS map_contains_key(map(1, a, 2, b), 5.0)#x] ++- OneRowRelation + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 1.0) +-- !query analysis +Project [map_contains_key(cast(map(1, a, 2, b) as map), cast(1.0 as decimal(11,1))) AS map_contains_key(map(1, a, 2, b), 1.0)#x] ++- OneRowRelation + + +-- !query +select map_contains_key(map(1.0, 'a', 2, 'b'), 5) +-- !query analysis +Project [map_contains_key(map(cast(1.0 as decimal(11,1)), a, cast(2 as decimal(11,1)), b), cast(5 as decimal(11,1))) AS map_contains_key(map(1.0, a, 2, b), 5)#x] ++- OneRowRelation + + +-- !query +select map_contains_key(map(1.0, 'a', 2, 'b'), 1) +-- !query analysis +Project [map_contains_key(map(cast(1.0 as decimal(11,1)), a, cast(2 as decimal(11,1)), b), cast(1 as decimal(11,1))) AS map_contains_key(map(1.0, a, 2, b), 1)#x] ++- OneRowRelation + + +-- !query +select map_contains_key(map('1', 'a', '2', 'b'), 1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "\"MAP\"", + "functionName" : "`map_contains_key`", + "leftType" : "\"MAP\"", + "rightType" : "\"INT\"", + "sqlExpr" : "\"map_contains_key(map(1, a, 2, b), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "map_contains_key(map('1', 'a', '2', 'b'), 1)" + } ] +} + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), '1') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "\"MAP\"", + "functionName" : "`map_contains_key`", + "leftType" : "\"MAP\"", + "rightType" : "\"STRING\"", + "sqlExpr" : "\"map_contains_key(map(1, a, 2, b), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "map_contains_key(map(1, 'a', 2, 'b'), '1')" + } ] +} 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/ansi/math.sql.out new file mode 100644 index 0000000000000..7eb7fcff356a4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/math.sql.out @@ -0,0 +1,433 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT round(25y, 1) +-- !query analysis +Project [round(25, 1) AS round(25, 1)#x] ++- OneRowRelation + + +-- !query +SELECT round(25y, 0) +-- !query analysis +Project [round(25, 0) AS round(25, 0)#x] ++- OneRowRelation + + +-- !query +SELECT round(25y, -1) +-- !query analysis +Project [round(25, -1) AS round(25, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(25y, -2) +-- !query analysis +Project [round(25, -2) AS round(25, -2)#x] ++- OneRowRelation + + +-- !query +SELECT round(25y, -3) +-- !query analysis +Project [round(25, -3) AS round(25, -3)#x] ++- OneRowRelation + + +-- !query +SELECT round(127y, -1) +-- !query analysis +Project [round(127, -1) AS round(127, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(-128y, -1) +-- !query analysis +Project [round(-128, -1) AS round(-128, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(525s, 1) +-- !query analysis +Project [round(525, 1) AS round(525, 1)#x] ++- OneRowRelation + + +-- !query +SELECT round(525s, 0) +-- !query analysis +Project [round(525, 0) AS round(525, 0)#x] ++- OneRowRelation + + +-- !query +SELECT round(525s, -1) +-- !query analysis +Project [round(525, -1) AS round(525, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(525s, -2) +-- !query analysis +Project [round(525, -2) AS round(525, -2)#x] ++- OneRowRelation + + +-- !query +SELECT round(525s, -3) +-- !query analysis +Project [round(525, -3) AS round(525, -3)#x] ++- OneRowRelation + + +-- !query +SELECT round(32767s, -1) +-- !query analysis +Project [round(32767, -1) AS round(32767, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(-32768s, -1) +-- !query analysis +Project [round(-32768, -1) AS round(-32768, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(525, 1) +-- !query analysis +Project [round(525, 1) AS round(525, 1)#x] ++- OneRowRelation + + +-- !query +SELECT round(525, 0) +-- !query analysis +Project [round(525, 0) AS round(525, 0)#x] ++- OneRowRelation + + +-- !query +SELECT round(525, -1) +-- !query analysis +Project [round(525, -1) AS round(525, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(525, -2) +-- !query analysis +Project [round(525, -2) AS round(525, -2)#x] ++- OneRowRelation + + +-- !query +SELECT round(525, -3) +-- !query analysis +Project [round(525, -3) AS round(525, -3)#x] ++- OneRowRelation + + +-- !query +SELECT round(2147483647, -1) +-- !query analysis +Project [round(2147483647, -1) AS round(2147483647, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(-2147483647, -1) +-- !query analysis +Project [round(-2147483647, -1) AS round(-2147483647, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(525L, 1) +-- !query analysis +Project [round(525, 1) AS round(525, 1)#xL] ++- OneRowRelation + + +-- !query +SELECT round(525L, 0) +-- !query analysis +Project [round(525, 0) AS round(525, 0)#xL] ++- OneRowRelation + + +-- !query +SELECT round(525L, -1) +-- !query analysis +Project [round(525, -1) AS round(525, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT round(525L, -2) +-- !query analysis +Project [round(525, -2) AS round(525, -2)#xL] ++- OneRowRelation + + +-- !query +SELECT round(525L, -3) +-- !query analysis +Project [round(525, -3) AS round(525, -3)#xL] ++- OneRowRelation + + +-- !query +SELECT round(9223372036854775807L, -1) +-- !query analysis +Project [round(9223372036854775807, -1) AS round(9223372036854775807, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT round(-9223372036854775808L, -1) +-- !query analysis +Project [round(-9223372036854775808, -1) AS round(-9223372036854775808, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT bround(25y, 1) +-- !query analysis +Project [bround(25, 1) AS bround(25, 1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(25y, 0) +-- !query analysis +Project [bround(25, 0) AS bround(25, 0)#x] ++- OneRowRelation + + +-- !query +SELECT bround(25y, -1) +-- !query analysis +Project [bround(25, -1) AS bround(25, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(25y, -2) +-- !query analysis +Project [bround(25, -2) AS bround(25, -2)#x] ++- OneRowRelation + + +-- !query +SELECT bround(25y, -3) +-- !query analysis +Project [bround(25, -3) AS bround(25, -3)#x] ++- OneRowRelation + + +-- !query +SELECT bround(127y, -1) +-- !query analysis +Project [bround(127, -1) AS bround(127, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(-128y, -1) +-- !query analysis +Project [bround(-128, -1) AS bround(-128, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525s, 1) +-- !query analysis +Project [bround(525, 1) AS bround(525, 1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525s, 0) +-- !query analysis +Project [bround(525, 0) AS bround(525, 0)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525s, -1) +-- !query analysis +Project [bround(525, -1) AS bround(525, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525s, -2) +-- !query analysis +Project [bround(525, -2) AS bround(525, -2)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525s, -3) +-- !query analysis +Project [bround(525, -3) AS bround(525, -3)#x] ++- OneRowRelation + + +-- !query +SELECT bround(32767s, -1) +-- !query analysis +Project [bround(32767, -1) AS bround(32767, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(-32768s, -1) +-- !query analysis +Project [bround(-32768, -1) AS bround(-32768, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525, 1) +-- !query analysis +Project [bround(525, 1) AS bround(525, 1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525, 0) +-- !query analysis +Project [bround(525, 0) AS bround(525, 0)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525, -1) +-- !query analysis +Project [bround(525, -1) AS bround(525, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525, -2) +-- !query analysis +Project [bround(525, -2) AS bround(525, -2)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525, -3) +-- !query analysis +Project [bround(525, -3) AS bround(525, -3)#x] ++- OneRowRelation + + +-- !query +SELECT bround(2147483647, -1) +-- !query analysis +Project [bround(2147483647, -1) AS bround(2147483647, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(-2147483647, -1) +-- !query analysis +Project [bround(-2147483647, -1) AS bround(-2147483647, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525L, 1) +-- !query analysis +Project [bround(525, 1) AS bround(525, 1)#xL] ++- OneRowRelation + + +-- !query +SELECT bround(525L, 0) +-- !query analysis +Project [bround(525, 0) AS bround(525, 0)#xL] ++- OneRowRelation + + +-- !query +SELECT bround(525L, -1) +-- !query analysis +Project [bround(525, -1) AS bround(525, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT bround(525L, -2) +-- !query analysis +Project [bround(525, -2) AS bround(525, -2)#xL] ++- OneRowRelation + + +-- !query +SELECT bround(525L, -3) +-- !query analysis +Project [bround(525, -3) AS bround(525, -3)#xL] ++- OneRowRelation + + +-- !query +SELECT bround(9223372036854775807L, -1) +-- !query analysis +Project [bround(9223372036854775807, -1) AS bround(9223372036854775807, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT bround(-9223372036854775808L, -1) +-- !query analysis +Project [bround(-9223372036854775808, -1) AS bround(-9223372036854775808, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT conv('100', 2, 10) +-- !query analysis +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, true) 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] ++- OneRowRelation + + +-- !query +SELECT conv('92233720368547758070', 10, 16) +-- !query analysis +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, true) 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] ++- 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/ansi/parse-schema-string.sql.out new file mode 100644 index 0000000000000..45fc3bd03a782 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/parse-schema-string.sql.out @@ -0,0 +1,27 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select from_csv('1', 'create INT') +-- !query analysis +Project [from_csv(StructField(create,IntegerType,true), 1, Some(America/Los_Angeles), None) AS from_csv(1)#x] ++- OneRowRelation + + +-- !query +select from_csv('1', 'cube INT') +-- !query analysis +Project [from_csv(StructField(cube,IntegerType,true), 1, Some(America/Los_Angeles), None) AS from_csv(1)#x] ++- OneRowRelation + + +-- !query +select from_json('{"create":1}', 'create INT') +-- !query analysis +Project [from_json(StructField(create,IntegerType,true), {"create":1}, Some(America/Los_Angeles)) AS from_json({"create":1})#x] ++- OneRowRelation + + +-- !query +select from_json('{"cube":1}', 'cube INT') +-- !query analysis +Project [from_json(StructField(cube,IntegerType,true), {"cube":1}, Some(America/Los_Angeles)) AS from_json({"cube":1})#x] ++- OneRowRelation 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/ansi/string-functions.sql.out new file mode 100644 index 0000000000000..9a66950d36508 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/string-functions.sql.out @@ -0,0 +1,1474 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select concat_ws() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 0", + "functionName" : "`concat_ws`" + } +} + + +-- !query +select format_string() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 0", + "functionName" : "`format_string`" + } +} + + +-- !query +select 'a' || 'b' || 'c' +-- !query analysis +Project [concat(concat(a, b), c) AS concat(concat(a, b), c)#x] ++- OneRowRelation + + +-- !query +select replace('abc', 'b', '123') +-- !query analysis +Project [replace(abc, b, 123) AS replace(abc, b, 123)#x] ++- OneRowRelation + + +-- !query +select replace('abc', 'b') +-- !query analysis +Project [replace(abc, b, ) AS replace(abc, b, )#x] ++- OneRowRelation + + +-- !query +select length(uuid()), (uuid() <> uuid()) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aaads', null) +-- !query analysis +Project [locate(bar, foobarbar, 1) AS locate(bar, foobarbar, 1)#x, position(cast(null as string), foobarbar, 1) AS position(NULL, foobarbar, 1)#x, position(aaads, cast(null as string), 1) AS position(aaads, NULL, 1)#x] ++- OneRowRelation + + +-- !query +select left("abcd", 2), left("abcd", 5), left("abcd", '2'), left("abcd", null) +-- !query analysis +Project [left(abcd, 2) AS left(abcd, 2)#x, left(abcd, 5) AS left(abcd, 5)#x, left(abcd, cast(2 as int)) AS left(abcd, 2)#x, left(abcd, cast(null as int)) AS left(abcd, NULL)#x] ++- OneRowRelation + + +-- !query +select left(null, -2) +-- !query analysis +Project [left(cast(null as string), -2) AS left(NULL, -2)#x] ++- OneRowRelation + + +-- !query +select left("abcd", -2), left("abcd", 0), left("abcd", 'a') +-- !query analysis +Project [left(abcd, -2) AS left(abcd, -2)#x, left(abcd, 0) AS left(abcd, 0)#x, left(abcd, cast(a as int)) AS left(abcd, a)#x] ++- OneRowRelation + + +-- !query +select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null) +-- !query analysis +Project [right(abcd, 2) AS right(abcd, 2)#x, right(abcd, 5) AS right(abcd, 5)#x, right(abcd, cast(2 as int)) AS right(abcd, 2)#x, right(abcd, cast(null as int)) AS right(abcd, NULL)#x] ++- OneRowRelation + + +-- !query +select right(null, -2) +-- !query analysis +Project [right(cast(null as string), -2) AS right(NULL, -2)#x] ++- OneRowRelation + + +-- !query +select right("abcd", -2), right("abcd", 0), right("abcd", 'a') +-- !query analysis +Project [right(abcd, -2) AS right(abcd, -2)#x, right(abcd, 0) AS right(abcd, 0)#x, right(abcd, cast(a as int)) AS right(abcd, a)#x] ++- OneRowRelation + + +-- !query +SELECT split('aa1cc2ee3', '[1-9]+') +-- !query analysis +Project [split(aa1cc2ee3, [1-9]+, -1) AS split(aa1cc2ee3, [1-9]+, -1)#x] ++- OneRowRelation + + +-- !query +SELECT split('aa1cc2ee3', '[1-9]+', 2) +-- !query analysis +Project [split(aa1cc2ee3, [1-9]+, 2) AS split(aa1cc2ee3, [1-9]+, 2)#x] ++- OneRowRelation + + +-- !query +SELECT split('hello', '') +-- !query analysis +Project [split(hello, , -1) AS split(hello, , -1)#x] ++- OneRowRelation + + +-- !query +SELECT split('', '') +-- !query analysis +Project [split(, , -1) AS split(, , -1)#x] ++- OneRowRelation + + +-- !query +SELECT split('abc', null) +-- !query analysis +Project [split(abc, cast(null as string), -1) AS split(abc, NULL, -1)#x] ++- OneRowRelation + + +-- !query +SELECT split(null, 'b') +-- !query analysis +Project [split(cast(null as string), b, -1) AS split(NULL, b, -1)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '.', 2) +-- !query analysis +Project [split_part(11.12.13, ., 2) AS split_part(11.12.13, ., 2)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '.', -1) +-- !query analysis +Project [split_part(11.12.13, ., -1) AS split_part(11.12.13, ., -1)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '.', -3) +-- !query analysis +Project [split_part(11.12.13, ., -3) AS split_part(11.12.13, ., -3)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '', 1) +-- !query analysis +Project [split_part(11.12.13, , 1) AS split_part(11.12.13, , 1)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11ab12ab13', 'ab', 1) +-- !query analysis +Project [split_part(11ab12ab13, ab, 1) AS split_part(11ab12ab13, ab, 1)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '.', 0) +-- !query analysis +Project [split_part(11.12.13, ., 0) AS split_part(11.12.13, ., 0)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '.', 4) +-- !query analysis +Project [split_part(11.12.13, ., 4) AS split_part(11.12.13, ., 4)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '.', 5) +-- !query analysis +Project [split_part(11.12.13, ., 5) AS split_part(11.12.13, ., 5)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '.', -5) +-- !query analysis +Project [split_part(11.12.13, ., -5) AS split_part(11.12.13, ., -5)#x] ++- OneRowRelation + + +-- !query +SELECT split_part(null, '.', 1) +-- !query analysis +Project [split_part(cast(null as string), ., 1) AS split_part(NULL, ., 1)#x] ++- OneRowRelation + + +-- !query +SELECT split_part(str, delimiter, partNum) FROM VALUES ('11.12.13', '.', 3) AS v1(str, delimiter, partNum) +-- !query analysis +Project [split_part(str#x, delimiter#x, partNum#x) AS split_part(str, delimiter, partNum)#x] ++- SubqueryAlias v1 + +- LocalRelation [str#x, delimiter#x, partNum#x] + + +-- !query +SELECT substr('Spark SQL', 5) +-- !query analysis +Project [substr(Spark SQL, 5, 2147483647) AS substr(Spark SQL, 5, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substr('Spark SQL', -3) +-- !query analysis +Project [substr(Spark SQL, -3, 2147483647) AS substr(Spark SQL, -3, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substr('Spark SQL', 5, 1) +-- !query analysis +Project [substr(Spark SQL, 5, 1) AS substr(Spark SQL, 5, 1)#x] ++- OneRowRelation + + +-- !query +SELECT substr('Spark SQL' from 5) +-- !query analysis +Project [substring(Spark SQL, 5, 2147483647) AS substring(Spark SQL, 5, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substr('Spark SQL' from -3) +-- !query analysis +Project [substring(Spark SQL, -3, 2147483647) AS substring(Spark SQL, -3, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substr('Spark SQL' from 5 for 1) +-- !query analysis +Project [substring(Spark SQL, 5, 1) AS substring(Spark SQL, 5, 1)#x] ++- OneRowRelation + + +-- !query +SELECT substring('Spark SQL', 5) +-- !query analysis +Project [substring(Spark SQL, 5, 2147483647) AS substring(Spark SQL, 5, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substring('Spark SQL', -3) +-- !query analysis +Project [substring(Spark SQL, -3, 2147483647) AS substring(Spark SQL, -3, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substring('Spark SQL', 5, 1) +-- !query analysis +Project [substring(Spark SQL, 5, 1) AS substring(Spark SQL, 5, 1)#x] ++- OneRowRelation + + +-- !query +SELECT substring('Spark SQL' from 5) +-- !query analysis +Project [substring(Spark SQL, 5, 2147483647) AS substring(Spark SQL, 5, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substring('Spark SQL' from -3) +-- !query analysis +Project [substring(Spark SQL, -3, 2147483647) AS substring(Spark SQL, -3, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substring('Spark SQL' from 5 for 1) +-- !query analysis +Project [substring(Spark SQL, 5, 1) AS substring(Spark SQL, 5, 1)#x] ++- OneRowRelation + + +-- !query +SELECT trim(" xyz "), ltrim(" xyz "), rtrim(" xyz ") +-- !query analysis +Project [trim( xyz , None) AS trim( xyz )#x, ltrim( xyz , None) AS ltrim( xyz )#x, rtrim( xyz , None) AS rtrim( xyz )#x] ++- OneRowRelation + + +-- !query +SELECT trim(BOTH 'xyz' FROM 'yxTomxx'), trim('xyz' FROM 'yxTomxx') +-- !query analysis +Project [trim(yxTomxx, Some(xyz)) AS TRIM(BOTH xyz FROM yxTomxx)#x, trim(yxTomxx, Some(xyz)) AS TRIM(BOTH xyz FROM yxTomxx)#x] ++- OneRowRelation + + +-- !query +SELECT trim(BOTH 'x' FROM 'xxxbarxxx'), trim('x' FROM 'xxxbarxxx') +-- !query analysis +Project [trim(xxxbarxxx, Some(x)) AS TRIM(BOTH x FROM xxxbarxxx)#x, trim(xxxbarxxx, Some(x)) AS TRIM(BOTH x FROM xxxbarxxx)#x] ++- OneRowRelation + + +-- !query +SELECT trim(LEADING 'xyz' FROM 'zzzytest') +-- !query analysis +Project [ltrim(zzzytest, Some(xyz)) AS TRIM(LEADING xyz FROM zzzytest)#x] ++- OneRowRelation + + +-- !query +SELECT trim(LEADING 'xyz' FROM 'zzzytestxyz') +-- !query analysis +Project [ltrim(zzzytestxyz, Some(xyz)) AS TRIM(LEADING xyz FROM zzzytestxyz)#x] ++- OneRowRelation + + +-- !query +SELECT trim(LEADING 'xy' FROM 'xyxXxyLAST WORD') +-- !query analysis +Project [ltrim(xyxXxyLAST WORD, Some(xy)) AS TRIM(LEADING xy FROM xyxXxyLAST WORD)#x] ++- OneRowRelation + + +-- !query +SELECT trim(TRAILING 'xyz' FROM 'testxxzx') +-- !query analysis +Project [rtrim(testxxzx, Some(xyz)) AS TRIM(TRAILING xyz FROM testxxzx)#x] ++- OneRowRelation + + +-- !query +SELECT trim(TRAILING 'xyz' FROM 'xyztestxxzx') +-- !query analysis +Project [rtrim(xyztestxxzx, Some(xyz)) AS TRIM(TRAILING xyz FROM xyztestxxzx)#x] ++- OneRowRelation + + +-- !query +SELECT trim(TRAILING 'xy' FROM 'TURNERyxXxy') +-- !query analysis +Project [rtrim(TURNERyxXxy, Some(xy)) AS TRIM(TRAILING xy FROM TURNERyxXxy)#x] ++- OneRowRelation + + +-- !query +SELECT btrim('xyxtrimyyx', 'xy') +-- !query analysis +Project [btrim(xyxtrimyyx, xy) AS btrim(xyxtrimyyx, xy)#x] ++- OneRowRelation + + +-- !query +SELECT btrim(encode(" xyz ", 'utf-8')) +-- !query analysis +Project [btrim(encode( xyz , utf-8)) AS btrim(encode( xyz , utf-8))#x] ++- OneRowRelation + + +-- !query +SELECT btrim(encode('yxTomxx', 'utf-8'), encode('xyz', 'utf-8')) +-- !query analysis +Project [btrim(encode(yxTomxx, utf-8), encode(xyz, utf-8)) AS btrim(encode(yxTomxx, utf-8), encode(xyz, utf-8))#x] ++- OneRowRelation + + +-- !query +SELECT btrim(encode('xxxbarxxx', 'utf-8'), encode('x', 'utf-8')) +-- !query analysis +Project [btrim(encode(xxxbarxxx, utf-8), encode(x, utf-8)) AS btrim(encode(xxxbarxxx, utf-8), encode(x, utf-8))#x] ++- OneRowRelation + + +-- !query +SELECT lpad('hi', 'invalid_length') +-- !query analysis +Project [lpad(hi, cast(invalid_length as int), ) AS lpad(hi, invalid_length, )#x] ++- OneRowRelation + + +-- !query +SELECT rpad('hi', 'invalid_length') +-- !query analysis +Project [rpad(hi, cast(invalid_length as int), ) AS rpad(hi, invalid_length, )#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex(''), 5)) +-- !query analysis +Project [hex(lpad(lpad, unhex(, false), 5, 0x00)) AS hex(lpad(unhex(), 5, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aabb'), 5)) +-- !query analysis +Project [hex(lpad(lpad, unhex(aabb, false), 5, 0x00)) AS hex(lpad(unhex(aabb), 5, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aabbcc'), 2)) +-- !query analysis +Project [hex(lpad(lpad, unhex(aabbcc, false), 2, 0x00)) AS hex(lpad(unhex(aabbcc), 2, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('123'), 2)) +-- !query analysis +Project [hex(lpad(lpad, unhex(123, false), 2, 0x00)) AS hex(lpad(unhex(123), 2, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('12345'), 2)) +-- !query analysis +Project [hex(lpad(lpad, unhex(12345, false), 2, 0x00)) AS hex(lpad(unhex(12345), 2, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex(''), 5, unhex('1f'))) +-- !query analysis +Project [hex(lpad(lpad, unhex(, false), 5, unhex(1f, false))) AS hex(lpad(unhex(), 5, unhex(1f)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aa'), 5, unhex('1f'))) +-- !query analysis +Project [hex(lpad(lpad, unhex(aa, false), 5, unhex(1f, false))) AS hex(lpad(unhex(aa), 5, unhex(1f)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aa'), 6, unhex('1f'))) +-- !query analysis +Project [hex(lpad(lpad, unhex(aa, false), 6, unhex(1f, false))) AS hex(lpad(unhex(aa), 6, unhex(1f)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex(''), 5, unhex('1f2e'))) +-- !query analysis +Project [hex(lpad(lpad, unhex(, false), 5, unhex(1f2e, false))) AS hex(lpad(unhex(), 5, unhex(1f2e)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aa'), 5, unhex('1f2e'))) +-- !query analysis +Project [hex(lpad(lpad, unhex(aa, false), 5, unhex(1f2e, false))) AS hex(lpad(unhex(aa), 5, unhex(1f2e)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aa'), 6, unhex('1f2e'))) +-- !query analysis +Project [hex(lpad(lpad, unhex(aa, false), 6, unhex(1f2e, false))) AS hex(lpad(unhex(aa), 6, unhex(1f2e)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex(''), 6, unhex(''))) +-- !query analysis +Project [hex(lpad(lpad, unhex(, false), 6, unhex(, false))) AS hex(lpad(unhex(), 6, unhex()))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aabbcc'), 6, unhex(''))) +-- !query analysis +Project [hex(lpad(lpad, unhex(aabbcc, false), 6, unhex(, false))) AS hex(lpad(unhex(aabbcc), 6, unhex()))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aabbcc'), 2, unhex('ff'))) +-- !query analysis +Project [hex(lpad(lpad, unhex(aabbcc, false), 2, unhex(ff, false))) AS hex(lpad(unhex(aabbcc), 2, unhex(ff)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex(''), 5)) +-- !query analysis +Project [hex(rpad(rpad, unhex(, false), 5, 0x00)) AS hex(rpad(unhex(), 5, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aabb'), 5)) +-- !query analysis +Project [hex(rpad(rpad, unhex(aabb, false), 5, 0x00)) AS hex(rpad(unhex(aabb), 5, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aabbcc'), 2)) +-- !query analysis +Project [hex(rpad(rpad, unhex(aabbcc, false), 2, 0x00)) AS hex(rpad(unhex(aabbcc), 2, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('123'), 2)) +-- !query analysis +Project [hex(rpad(rpad, unhex(123, false), 2, 0x00)) AS hex(rpad(unhex(123), 2, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('12345'), 2)) +-- !query analysis +Project [hex(rpad(rpad, unhex(12345, false), 2, 0x00)) AS hex(rpad(unhex(12345), 2, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex(''), 5, unhex('1f'))) +-- !query analysis +Project [hex(rpad(rpad, unhex(, false), 5, unhex(1f, false))) AS hex(rpad(unhex(), 5, unhex(1f)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aa'), 5, unhex('1f'))) +-- !query analysis +Project [hex(rpad(rpad, unhex(aa, false), 5, unhex(1f, false))) AS hex(rpad(unhex(aa), 5, unhex(1f)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aa'), 6, unhex('1f'))) +-- !query analysis +Project [hex(rpad(rpad, unhex(aa, false), 6, unhex(1f, false))) AS hex(rpad(unhex(aa), 6, unhex(1f)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex(''), 5, unhex('1f2e'))) +-- !query analysis +Project [hex(rpad(rpad, unhex(, false), 5, unhex(1f2e, false))) AS hex(rpad(unhex(), 5, unhex(1f2e)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aa'), 5, unhex('1f2e'))) +-- !query analysis +Project [hex(rpad(rpad, unhex(aa, false), 5, unhex(1f2e, false))) AS hex(rpad(unhex(aa), 5, unhex(1f2e)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aa'), 6, unhex('1f2e'))) +-- !query analysis +Project [hex(rpad(rpad, unhex(aa, false), 6, unhex(1f2e, false))) AS hex(rpad(unhex(aa), 6, unhex(1f2e)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex(''), 6, unhex(''))) +-- !query analysis +Project [hex(rpad(rpad, unhex(, false), 6, unhex(, false))) AS hex(rpad(unhex(), 6, unhex()))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aabbcc'), 6, unhex(''))) +-- !query analysis +Project [hex(rpad(rpad, unhex(aabbcc, false), 6, unhex(, false))) AS hex(rpad(unhex(aabbcc), 6, unhex()))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aabbcc'), 2, unhex('ff'))) +-- !query analysis +Project [hex(rpad(rpad, unhex(aabbcc, false), 2, unhex(ff, false))) AS hex(rpad(unhex(aabbcc), 2, unhex(ff)))#x] ++- OneRowRelation + + +-- !query +SELECT lpad('abc', 5, x'57') +-- !query analysis +Project [lpad(abc, 5, cast(0x57 as string)) AS lpad(abc, 5, X'57')#x] ++- OneRowRelation + + +-- !query +SELECT lpad(x'57', 5, 'abc') +-- !query analysis +Project [lpad(cast(0x57 as string), 5, abc) AS lpad(X'57', 5, abc)#x] ++- OneRowRelation + + +-- !query +SELECT rpad('abc', 5, x'57') +-- !query analysis +Project [rpad(abc, 5, cast(0x57 as string)) AS rpad(abc, 5, X'57')#x] ++- OneRowRelation + + +-- !query +SELECT rpad(x'57', 5, 'abc') +-- !query analysis +Project [rpad(cast(0x57 as string), 5, abc) AS rpad(X'57', 5, abc)#x] ++- OneRowRelation + + +-- !query +select decode() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "2", + "functionName" : "`decode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "decode()" + } ] +} + + +-- !query +select decode(encode('abc', 'utf-8')) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "2", + "functionName" : "`decode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "decode(encode('abc', 'utf-8'))" + } ] +} + + +-- !query +select decode(encode('abc', 'utf-8'), 'utf-8') +-- !query analysis +Project [decode(encode(abc, utf-8), utf-8) AS decode(encode(abc, utf-8), utf-8)#x] ++- OneRowRelation + + +-- !query +select decode(1, 1, 'Southlake') +-- !query analysis +Project [decode(1, 1, Southlake) AS decode(1, 1, Southlake)#x] ++- OneRowRelation + + +-- !query +select decode(2, 1, 'Southlake') +-- !query analysis +Project [decode(2, 1, Southlake) AS decode(2, 1, Southlake)#x] ++- OneRowRelation + + +-- !query +select decode(2, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle', 'Non domestic') +-- !query analysis +Project [decode(2, 1, Southlake, 2, San Francisco, 3, New Jersey, 4, Seattle, Non domestic) AS decode(2, 1, Southlake, 2, San Francisco, 3, New Jersey, 4, Seattle, Non domestic)#x] ++- OneRowRelation + + +-- !query +select decode(6, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle', 'Non domestic') +-- !query analysis +Project [decode(6, 1, Southlake, 2, San Francisco, 3, New Jersey, 4, Seattle, Non domestic) AS decode(6, 1, Southlake, 2, San Francisco, 3, New Jersey, 4, Seattle, Non domestic)#x] ++- OneRowRelation + + +-- !query +select decode(6, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle') +-- !query analysis +Project [decode(6, 1, Southlake, 2, San Francisco, 3, New Jersey, 4, Seattle) AS decode(6, 1, Southlake, 2, San Francisco, 3, New Jersey, 4, Seattle)#x] ++- OneRowRelation + + +-- !query +select decode(null, 6, 'Spark', NULL, 'SQL', 4, 'rocks') +-- !query analysis +Project [decode(null, 6, Spark, null, SQL, 4, rocks) AS decode(NULL, 6, Spark, NULL, SQL, 4, rocks)#x] ++- OneRowRelation + + +-- !query +select decode(null, 6, 'Spark', NULL, 'SQL', 4, 'rocks', NULL, '.') +-- !query analysis +Project [decode(null, 6, Spark, null, SQL, 4, rocks, null, .) AS decode(NULL, 6, Spark, NULL, SQL, 4, rocks, NULL, .)#x] ++- OneRowRelation + + +-- !query +SELECT CONTAINS(null, 'Spark') +-- !query analysis +Project [Contains(cast(null as string), Spark) AS contains(NULL, Spark)#x] ++- OneRowRelation + + +-- !query +SELECT CONTAINS('Spark SQL', null) +-- !query analysis +Project [Contains(Spark SQL, cast(null as string)) AS contains(Spark SQL, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT CONTAINS(null, null) +-- !query analysis +Project [Contains(cast(null as string), cast(null as string)) AS contains(NULL, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT CONTAINS('Spark SQL', 'Spark') +-- !query analysis +Project [Contains(Spark SQL, Spark) AS contains(Spark SQL, Spark)#x] ++- OneRowRelation + + +-- !query +SELECT CONTAINS('Spark SQL', 'SQL') +-- !query analysis +Project [Contains(Spark SQL, SQL) AS contains(Spark SQL, SQL)#x] ++- OneRowRelation + + +-- !query +SELECT CONTAINS('Spark SQL', 'SPARK') +-- !query analysis +Project [Contains(Spark SQL, SPARK) AS contains(Spark SQL, SPARK)#x] ++- OneRowRelation + + +-- !query +SELECT startswith('Spark SQL', 'ark') +-- !query analysis +Project [StartsWith(Spark SQL, ark) AS startswith(Spark SQL, ark)#x] ++- OneRowRelation + + +-- !query +SELECT startswith('Spark SQL', 'Spa') +-- !query analysis +Project [StartsWith(Spark SQL, Spa) AS startswith(Spark SQL, Spa)#x] ++- OneRowRelation + + +-- !query +SELECT startswith(null, 'Spark') +-- !query analysis +Project [StartsWith(cast(null as string), Spark) AS startswith(NULL, Spark)#x] ++- OneRowRelation + + +-- !query +SELECT startswith('Spark', null) +-- !query analysis +Project [StartsWith(Spark, cast(null as string)) AS startswith(Spark, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT startswith(null, null) +-- !query analysis +Project [StartsWith(cast(null as string), cast(null as string)) AS startswith(NULL, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT endswith('Spark SQL', 'QL') +-- !query analysis +Project [EndsWith(Spark SQL, QL) AS endswith(Spark SQL, QL)#x] ++- OneRowRelation + + +-- !query +SELECT endswith('Spark SQL', 'Spa') +-- !query analysis +Project [EndsWith(Spark SQL, Spa) AS endswith(Spark SQL, Spa)#x] ++- OneRowRelation + + +-- !query +SELECT endswith(null, 'Spark') +-- !query analysis +Project [EndsWith(cast(null as string), Spark) AS endswith(NULL, Spark)#x] ++- OneRowRelation + + +-- !query +SELECT endswith('Spark', null) +-- !query analysis +Project [EndsWith(Spark, cast(null as string)) AS endswith(Spark, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT endswith(null, null) +-- !query analysis +Project [EndsWith(cast(null as string), cast(null as string)) AS endswith(NULL, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT contains(x'537061726b2053514c', x'537061726b') +-- !query analysis +Project [contains(contains, 0x537061726B2053514C, 0x537061726B) AS contains(X'537061726B2053514C', X'537061726B')#x] ++- OneRowRelation + + +-- !query +SELECT contains(x'', x'') +-- !query analysis +Project [contains(contains, 0x, 0x) AS contains(X'', X'')#x] ++- OneRowRelation + + +-- !query +SELECT contains(x'537061726b2053514c', null) +-- !query analysis +Project [Contains(cast(0x537061726B2053514C as string), cast(null as string)) AS contains(X'537061726B2053514C', NULL)#x] ++- OneRowRelation + + +-- !query +SELECT contains(12, '1') +-- !query analysis +Project [Contains(cast(12 as string), 1) AS contains(12, 1)#x] ++- OneRowRelation + + +-- !query +SELECT contains(true, 'ru') +-- !query analysis +Project [Contains(cast(true as string), ru) AS contains(true, ru)#x] ++- OneRowRelation + + +-- !query +SELECT contains(x'12', 12) +-- !query analysis +Project [Contains(cast(0x12 as string), cast(12 as string)) AS contains(X'12', 12)#x] ++- OneRowRelation + + +-- !query +SELECT contains(true, false) +-- !query analysis +Project [Contains(cast(true as string), cast(false as string)) AS contains(true, false)#x] ++- OneRowRelation + + +-- !query +SELECT startswith(x'537061726b2053514c', x'537061726b') +-- !query analysis +Project [startswith(startswith, 0x537061726B2053514C, 0x537061726B) AS startswith(X'537061726B2053514C', X'537061726B')#x] ++- OneRowRelation + + +-- !query +SELECT startswith(x'537061726b2053514c', x'') +-- !query analysis +Project [startswith(startswith, 0x537061726B2053514C, 0x) AS startswith(X'537061726B2053514C', X'')#x] ++- OneRowRelation + + +-- !query +SELECT startswith(x'', x'') +-- !query analysis +Project [startswith(startswith, 0x, 0x) AS startswith(X'', X'')#x] ++- OneRowRelation + + +-- !query +SELECT startswith(x'537061726b2053514c', null) +-- !query analysis +Project [StartsWith(cast(0x537061726B2053514C as string), cast(null as string)) AS startswith(X'537061726B2053514C', NULL)#x] ++- OneRowRelation + + +-- !query +SELECT endswith(x'537061726b2053514c', x'53516c') +-- !query analysis +Project [endswith(endswith, 0x537061726B2053514C, 0x53516C) AS endswith(X'537061726B2053514C', X'53516C')#x] ++- OneRowRelation + + +-- !query +SELECT endsWith(x'537061726b2053514c', x'537061726b') +-- !query analysis +Project [endswith(endswith, 0x537061726B2053514C, 0x537061726B) AS endswith(X'537061726B2053514C', X'537061726B')#x] ++- OneRowRelation + + +-- !query +SELECT endsWith(x'537061726b2053514c', x'') +-- !query analysis +Project [endswith(endswith, 0x537061726B2053514C, 0x) AS endswith(X'537061726B2053514C', X'')#x] ++- OneRowRelation + + +-- !query +SELECT endsWith(x'', x'') +-- !query analysis +Project [endswith(endswith, 0x, 0x) AS endswith(X'', X'')#x] ++- OneRowRelation + + +-- !query +SELECT endsWith(x'537061726b2053514c', null) +-- !query analysis +Project [EndsWith(cast(0x537061726B2053514C as string), cast(null as string)) AS endswith(X'537061726B2053514C', NULL)#x] ++- OneRowRelation + + +-- !query +select to_number('454', '000') +-- !query analysis +Project [to_number(454, 000) AS to_number(454, 000)#x] ++- OneRowRelation + + +-- !query +select to_number('454.2', '000.0') +-- !query analysis +Project [to_number(454.2, 000.0) AS to_number(454.2, 000.0)#x] ++- OneRowRelation + + +-- !query +select to_number('12,454', '00,000') +-- !query analysis +Project [to_number(12,454, 00,000) AS to_number(12,454, 00,000)#x] ++- OneRowRelation + + +-- !query +select to_number('$78.12', '$00.00') +-- !query analysis +Project [to_number($78.12, $00.00) AS to_number($78.12, $00.00)#x] ++- OneRowRelation + + +-- !query +select to_number('+454', 'S000') +-- !query analysis +Project [to_number(+454, S000) AS to_number(+454, S000)#x] ++- OneRowRelation + + +-- !query +select to_number('-454', 'S000') +-- !query analysis +Project [to_number(-454, S000) AS to_number(-454, S000)#x] ++- OneRowRelation + + +-- !query +select to_number('12,454.8-', '00,000.9MI') +-- !query analysis +Project [to_number(12,454.8-, 00,000.9MI) AS to_number(12,454.8-, 00,000.9MI)#x] ++- OneRowRelation + + +-- !query +select to_number('00,454.8-', '00,000.9MI') +-- !query analysis +Project [to_number(00,454.8-, 00,000.9MI) AS to_number(00,454.8-, 00,000.9MI)#x] ++- OneRowRelation + + +-- !query +select to_number('<00,454.8>', '00,000.9PR') +-- !query analysis +Project [to_number(<00,454.8>, 00,000.9PR) AS to_number(<00,454.8>, 00,000.9PR)#x] ++- OneRowRelation + + +-- !query +select to_binary('', 'base64') +-- !query analysis +Project [to_binary(, Some(base64), false) AS to_binary(, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary(' ', 'base64') +-- !query analysis +Project [to_binary( , Some(base64), false) AS to_binary( , base64)#x] ++- OneRowRelation + + +-- !query +select to_binary(' ab cd ', 'base64') +-- !query analysis +Project [to_binary( ab cd , Some(base64), false) AS to_binary( ab cd , base64)#x] ++- OneRowRelation + + +-- !query +select to_binary(' ab c=', 'base64') +-- !query analysis +Project [to_binary( ab c=, Some(base64), false) AS to_binary( ab c=, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary(' ab cdef= = ', 'base64') +-- !query analysis +Project [to_binary( ab cdef= = , Some(base64), false) AS to_binary( ab cdef= = , base64)#x] ++- OneRowRelation + + +-- !query +select to_binary( + concat(' b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0', + 'd2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl'), 'base64') +-- !query analysis +Project [to_binary(concat( b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0, d2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl), Some(base64), false) AS to_binary(concat( b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0, d2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl), base64)#x] ++- OneRowRelation + + +-- !query +select to_binary('a', 'base64') +-- !query analysis +Project [to_binary(a, Some(base64), false) AS to_binary(a, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary('a?', 'base64') +-- !query analysis +Project [to_binary(a?, Some(base64), false) AS to_binary(a?, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary('abcde', 'base64') +-- !query analysis +Project [to_binary(abcde, Some(base64), false) AS to_binary(abcde, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary('abcd=', 'base64') +-- !query analysis +Project [to_binary(abcd=, Some(base64), false) AS to_binary(abcd=, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary('a===', 'base64') +-- !query analysis +Project [to_binary(a===, Some(base64), false) AS to_binary(a===, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary('ab==f', 'base64') +-- !query analysis +Project [to_binary(ab==f, Some(base64), false) AS to_binary(ab==f, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary( + '∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β)', 'utf-8') +-- !query analysis +Project [to_binary(∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β), Some(utf-8), false) AS to_binary(∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β), utf-8)#x] ++- OneRowRelation + + +-- !query +select to_binary('大千世界', 'utf8') +-- !query analysis +Project [to_binary(大千世界, Some(utf8), false) AS to_binary(大千世界, utf8)#x] ++- OneRowRelation + + +-- !query +select to_binary('', 'utf-8') +-- !query analysis +Project [to_binary(, Some(utf-8), false) AS to_binary(, utf-8)#x] ++- OneRowRelation + + +-- !query +select to_binary(' ', 'utf8') +-- !query analysis +Project [to_binary( , Some(utf8), false) AS to_binary( , utf8)#x] ++- OneRowRelation + + +-- !query +select to_binary('737472696E67') +-- !query analysis +Project [to_binary(737472696E67, None, false) AS to_binary(737472696E67)#x] ++- OneRowRelation + + +-- !query +select to_binary('737472696E67', 'hex') +-- !query analysis +Project [to_binary(737472696E67, Some(hex), false) AS to_binary(737472696E67, hex)#x] ++- OneRowRelation + + +-- !query +select to_binary('') +-- !query analysis +Project [to_binary(, None, false) AS to_binary()#x] ++- OneRowRelation + + +-- !query +select to_binary('1', 'hex') +-- !query analysis +Project [to_binary(1, Some(hex), false) AS to_binary(1, hex)#x] ++- OneRowRelation + + +-- !query +select to_binary('FF') +-- !query analysis +Project [to_binary(FF, None, false) AS to_binary(FF)#x] ++- OneRowRelation + + +-- !query +select to_binary('123', 'hex') +-- !query analysis +Project [to_binary(123, Some(hex), false) AS to_binary(123, hex)#x] ++- OneRowRelation + + +-- !query +select to_binary('12345', 'hex') +-- !query analysis +Project [to_binary(12345, Some(hex), false) AS to_binary(12345, hex)#x] ++- OneRowRelation + + +-- !query +select to_binary('GG') +-- !query analysis +Project [to_binary(GG, None, false) AS to_binary(GG)#x] ++- OneRowRelation + + +-- !query +select to_binary('01 AF', 'hex') +-- !query analysis +Project [to_binary(01 AF, Some(hex), false) AS to_binary(01 AF, hex)#x] ++- OneRowRelation + + +-- !query +select to_binary('abc', concat('utf', '-8')) +-- !query analysis +Project [to_binary(abc, Some(concat(utf, -8)), false) AS to_binary(abc, concat(utf, -8))#x] ++- OneRowRelation + + +-- !query +select to_binary(' ab cdef= = ', substr('base64whynot', 0, 6)) +-- !query analysis +Project [to_binary( ab cdef= = , Some(substr(base64whynot, 0, 6)), false) AS to_binary( ab cdef= = , substr(base64whynot, 0, 6))#x] ++- OneRowRelation + + +-- !query +select to_binary(' ab cdef= = ', replace('HEX0', '0')) +-- !query analysis +Project [to_binary( ab cdef= = , Some(replace(HEX0, 0, )), false) AS to_binary( ab cdef= = , replace(HEX0, 0, ))#x] ++- OneRowRelation + + +-- !query +select to_binary('abc', 'Hex') +-- !query analysis +Project [to_binary(abc, Some(Hex), false) AS to_binary(abc, Hex)#x] ++- OneRowRelation + + +-- !query +select to_binary('abc', null) +-- !query analysis +Project [to_binary(abc, Some(cast(null as string)), false) AS to_binary(abc, NULL)#x] ++- OneRowRelation + + +-- !query +select to_binary(null, 'utf-8') +-- !query analysis +Project [to_binary(cast(null as string), Some(utf-8), false) AS to_binary(NULL, utf-8)#x] ++- OneRowRelation + + +-- !query +select to_binary(null, null) +-- !query analysis +Project [to_binary(cast(null as string), Some(cast(null as string)), false) AS to_binary(NULL, NULL)#x] ++- OneRowRelation + + +-- !query +select to_binary(null, cast(null as string)) +-- !query analysis +Project [to_binary(cast(null as string), Some(cast(null as string)), false) AS to_binary(NULL, CAST(NULL AS STRING))#x] ++- OneRowRelation + + +-- !query +select to_binary('abc', 1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", + "sqlState" : "42K09", + "messageParameters" : { + "inputName" : "fmt", + "inputValue" : "'1'", + "requireType" : "case-insensitive \"STRING\"", + "sqlExpr" : "\"to_binary(abc, 1)\"", + "validValues" : "'hex', 'utf-8', 'utf8', or 'base64'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "to_binary('abc', 1)" + } ] +} + + +-- !query +select to_binary('abc', 'invalidFormat') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", + "sqlState" : "42K09", + "messageParameters" : { + "inputName" : "fmt", + "inputValue" : "'invalidformat'", + "requireType" : "case-insensitive \"STRING\"", + "sqlExpr" : "\"to_binary(abc, invalidFormat)\"", + "validValues" : "'hex', 'utf-8', 'utf8', or 'base64'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "to_binary('abc', 'invalidFormat')" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW fmtTable(fmtField) AS SELECT * FROM VALUES ('invalidFormat') +-- !query analysis +CreateViewCommand `fmtTable`, [(fmtField,None)], SELECT * FROM VALUES ('invalidFormat'), false, false, LocalTempView, true + +- Project [col1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT to_binary('abc', fmtField) FROM fmtTable +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"fmtField\"", + "inputName" : "fmt", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"to_binary(abc, fmtField)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "to_binary('abc', fmtField)" + } ] +} + + +-- !query +DROP VIEW IF EXISTS fmtTable +-- !query analysis +DropTempViewCommand fmtTable + + +-- !query +select luhn_check('4111111111111111') +-- !query analysis +Project [luhn_check(4111111111111111) AS luhn_check(4111111111111111)#x] ++- OneRowRelation + + +-- !query +select luhn_check('5500000000000004') +-- !query analysis +Project [luhn_check(5500000000000004) AS luhn_check(5500000000000004)#x] ++- OneRowRelation + + +-- !query +select luhn_check('340000000000009') +-- !query analysis +Project [luhn_check(340000000000009) AS luhn_check(340000000000009)#x] ++- OneRowRelation + + +-- !query +select luhn_check('6011000000000004') +-- !query analysis +Project [luhn_check(6011000000000004) AS luhn_check(6011000000000004)#x] ++- OneRowRelation + + +-- !query +select luhn_check('6011000000000005') +-- !query analysis +Project [luhn_check(6011000000000005) AS luhn_check(6011000000000005)#x] ++- OneRowRelation + + +-- !query +select luhn_check('378282246310006') +-- !query analysis +Project [luhn_check(378282246310006) AS luhn_check(378282246310006)#x] ++- OneRowRelation + + +-- !query +select luhn_check('0') +-- !query analysis +Project [luhn_check(0) AS luhn_check(0)#x] ++- OneRowRelation + + +-- !query +select luhn_check('4111111111111111 ') +-- !query analysis +Project [luhn_check(4111111111111111 ) AS luhn_check(4111111111111111 )#x] ++- OneRowRelation + + +-- !query +select luhn_check('4111111 111111111') +-- !query analysis +Project [luhn_check(4111111 111111111) AS luhn_check(4111111 111111111)#x] ++- OneRowRelation + + +-- !query +select luhn_check(' 4111111111111111') +-- !query analysis +Project [luhn_check( 4111111111111111) AS luhn_check( 4111111111111111)#x] ++- OneRowRelation + + +-- !query +select luhn_check('') +-- !query analysis +Project [luhn_check() AS luhn_check()#x] ++- OneRowRelation + + +-- !query +select luhn_check(' ') +-- !query analysis +Project [luhn_check( ) AS luhn_check( )#x] ++- OneRowRelation + + +-- !query +select luhn_check('510B105105105106') +-- !query analysis +Project [luhn_check(510B105105105106) AS luhn_check(510B105105105106)#x] ++- OneRowRelation + + +-- !query +select luhn_check('ABCDED') +-- !query analysis +Project [luhn_check(ABCDED) AS luhn_check(ABCDED)#x] ++- OneRowRelation + + +-- !query +select luhn_check(null) +-- !query analysis +Project [luhn_check(cast(null as string)) AS luhn_check(NULL)#x] ++- OneRowRelation + + +-- !query +select luhn_check(6011111111111117) +-- !query analysis +Project [luhn_check(cast(6011111111111117 as string)) AS luhn_check(6011111111111117)#x] ++- OneRowRelation + + +-- !query +select luhn_check(6011111111111118) +-- !query analysis +Project [luhn_check(cast(6011111111111118 as string)) AS luhn_check(6011111111111118)#x] ++- OneRowRelation + + +-- !query +select luhn_check(123.456) +-- !query analysis +Project [luhn_check(cast(123.456 as string)) AS luhn_check(123.456)#x] ++- OneRowRelation 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/ansi/timestamp.sql.out new file mode 100644 index 0000000000000..65baeb48e1167 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/timestamp.sql.out @@ -0,0 +1,789 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select timestamp '2019-01-01\t' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp '2019-01-01中文' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2019-01-01中文'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} + + +-- !query +select timestamp'4294967297' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'4294967297'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} + + +-- !query +select timestamp'2021-01-01T12:30:4294967297.123456' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-01-01T12:30:4294967297.123456'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} + + +-- !query +select current_timestamp = current_timestamp +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select current_timestamp() = current_timestamp() +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select localtimestamp() = localtimestamp() +-- !query analysis +Project [(localtimestamp(Some(America/Los_Angeles)) = localtimestamp(Some(America/Los_Angeles))) AS (localtimestamp() = localtimestamp())#x] ++- OneRowRelation + + +-- !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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) +-- !query analysis +Project [timestamp_seconds(1230219000) AS timestamp_seconds(1230219000)#x, timestamp_seconds(-1230219000) AS timestamp_seconds(-1230219000)#x, timestamp_seconds(null) AS timestamp_seconds(NULL)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)) +-- !query analysis +Project [timestamp_seconds(1.23) AS timestamp_seconds(1.23)#x, timestamp_seconds(1.23) AS timestamp_seconds(1.23)#x, timestamp_seconds(cast(1.23 as float)) AS timestamp_seconds(1.23)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null) +-- !query analysis +Project [timestamp_millis(1230219000123) AS timestamp_millis(1230219000123)#x, timestamp_millis(-1230219000123) AS timestamp_millis(-1230219000123)#x, timestamp_millis(null) AS timestamp_millis(NULL)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null) +-- !query analysis +Project [timestamp_micros(1230219000123123) AS timestamp_micros(1230219000123123)#x, timestamp_micros(-1230219000123123) AS timestamp_micros(-1230219000123123)#x, timestamp_micros(null) AS timestamp_micros(NULL)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(1230219000123123) +-- !query analysis +Project [timestamp_seconds(1230219000123123) AS timestamp_seconds(1230219000123123)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(-1230219000123123) +-- !query analysis +Project [timestamp_seconds(-1230219000123123) AS timestamp_seconds(-1230219000123123)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MILLIS(92233720368547758) +-- !query analysis +Project [timestamp_millis(92233720368547758) AS timestamp_millis(92233720368547758)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MILLIS(-92233720368547758) +-- !query analysis +Project [timestamp_millis(-92233720368547758) AS timestamp_millis(-92233720368547758)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(0.1234567) +-- !query analysis +Project [timestamp_seconds(0.1234567) AS timestamp_seconds(0.1234567)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)) +-- !query analysis +Project [timestamp_seconds(0.1234567) AS timestamp_seconds(0.1234567)#x, timestamp_seconds(cast(0.1234567 as float)) AS timestamp_seconds(0.1234567)#x] ++- OneRowRelation + + +-- !query +create temporary view ttf1 as select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`) +-- !query analysis +CreateViewCommand `ttf1`, select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`), false, false, LocalTempView, true + +- Project [current_date#x, current_timestamp#x] + +- SubqueryAlias ttf1 + +- LocalRelation [current_date#x, current_timestamp#x] + + +-- !query +select typeof(current_date), typeof(current_timestamp) from ttf1 +-- !query analysis +Project [typeof(current_date#x) AS typeof(current_date)#x, typeof(current_timestamp#x) AS typeof(current_timestamp)#x] ++- SubqueryAlias ttf1 + +- View (`ttf1`, [current_date#x,current_timestamp#x]) + +- Project [cast(current_date#x as int) AS current_date#x, cast(current_timestamp#x as int) AS current_timestamp#x] + +- Project [current_date#x, current_timestamp#x] + +- SubqueryAlias ttf1 + +- LocalRelation [current_date#x, current_timestamp#x] + + +-- !query +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b) +-- !query analysis +CreateViewCommand `ttf2`, select * from values + (1, 2), + (2, 3) + as ttf2(a, b), false, false, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias ttf2 + +- LocalRelation [a#x, b#x] + + +-- !query +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2 +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select a, b from ttf2 order by a, current_date +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- OneRowRelation + + +-- !query +select to_timestamp(1) +-- !query analysis +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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), true) AS to_timestamp(2019 10:10:10, yyyy hh:mm:ss)#x] ++- OneRowRelation + + +-- !query +select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp'2011-11-11 11:11:11' - null +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select null - timestamp'2011-11-11 11:11:11' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +create temporary view ts_view as select '2011-11-11 11:11:11' str +-- !query analysis +CreateViewCommand `ts_view`, select '2011-11-11 11:11:11' str, false, false, LocalTempView, true + +- Project [2011-11-11 11:11:11 AS str#x] + +- OneRowRelation + + +-- !query +select str - timestamp'2011-11-11 11:11:11' from ts_view +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp'2011-11-11 11:11:11' - str from ts_view +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp'2011-11-11 11:11:11' + '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "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" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" + } ] +} + + +-- !query +select '1' + timestamp'2011-11-11 11:11:11' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "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" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + null +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "timestamp'2011-11-11 11:11:11' + null" + } ] +} + + +-- !query +select null + timestamp'2011-11-11 11:11:11' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "null + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + interval '2' day, + timestamp'2011-11-11 11:11:11' - interval '2-2' year to month, + timestamp'2011-11-11 11:11:11' + interval '-2' second, + timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second, + - interval 2 years + timestamp'2011-11-11 11:11:11', + interval '1 12' day to hour + timestamp'2011-11-11 11:11:11' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2012-01-01' - interval 3 hours, + date '2012-01-01' + interval '12:12:12' hour to second, + interval '2' minute + date '2012-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- 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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_json(StructField(t,TimestampType,true), (timestampFormat,dd/MMMMM/yyyy), {"t":"26/October/2015"}, Some(America/Los_Angeles)) AS from_json({"t":"26/October/2015"})#x] ++- OneRowRelation + + +-- !query +select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_csv(StructField(t,TimestampType,true), (timestampFormat,dd/MMMMM/yyyy), 26/October/2015, Some(America/Los_Angeles), None) AS from_csv(26/October/2015)#x] ++- OneRowRelation + + +-- !query +select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampadd(YEAR, 1, date'2022-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampadd(SECOND, -1, date'2022-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] 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/ansi/try_aggregates.sql.out new file mode 100644 index 0000000000000..734f6a6139623 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_aggregates.sql.out @@ -0,0 +1,255 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_sum(col) FROM VALUES (5), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#xL] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#xL] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (NULL), (NULL) AS tab(col) +-- !query analysis +Aggregate [try_sum(cast(col#x as double)) AS try_sum(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query analysis +Aggregate [try_sum(col#xL) AS try_sum(col)#xL] ++- SubqueryAlias tab + +- LocalRelation [col#xL] + + +-- !query +SELECT try_sum(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (5), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_sum((cast(col#x as double) / cast(0 as double))) AS try_sum((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query analysis +Aggregate [try_sum((col#x / cast(0 as decimal(1,0)))) AS try_sum((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_sum((cast(col#x as double) / cast(0 as double))) AS try_sum((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query analysis +Aggregate [try_sum((col#xL + 1)) AS try_sum((col + 1))#xL] ++- SubqueryAlias tab + +- LocalRelation [col#xL] + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query analysis +Aggregate [try_sum((col#x / 0)) AS try_sum((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query analysis +Aggregate [try_sum((col#x / 0)) AS try_sum((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (5), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (NULL), (NULL) AS tab(col) +-- !query analysis +Aggregate [try_avg(cast(col#x as double)) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query analysis +Aggregate [try_avg(col#xL) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#xL] + + +-- !query +SELECT try_avg(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (5), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_avg((cast(col#x as double) / cast(0 as double))) AS try_avg((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query analysis +Aggregate [try_avg((col#x / cast(0 as decimal(1,0)))) AS try_avg((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_avg((cast(col#x as double) / cast(0 as double))) AS try_avg((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query analysis +Aggregate [try_avg((col#xL + 1)) AS try_avg((col + 1))#x] ++- SubqueryAlias tab + +- LocalRelation [col#xL] + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query analysis +Aggregate [try_avg((col#x / 0)) AS try_avg((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query analysis +Aggregate [try_avg((col#x / 0)) AS try_avg((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] 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/ansi/try_arithmetic.sql.out new file mode 100644 index 0000000000000..e8506a8589d8c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_arithmetic.sql.out @@ -0,0 +1,435 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_add(1, 1) +-- !query analysis +Project [try_add(1, 1) AS try_add(1, 1)#x] ++- OneRowRelation + + +-- !query +SELECT try_add(2147483647, 1) +-- !query analysis +Project [try_add(2147483647, 1) AS try_add(2147483647, 1)#x] ++- OneRowRelation + + +-- !query +SELECT try_add(-2147483648, -1) +-- !query analysis +Project [try_add(-2147483648, -1) AS try_add(-2147483648, -1)#x] ++- OneRowRelation + + +-- !query +SELECT try_add(9223372036854775807L, 1) +-- !query analysis +Project [try_add(9223372036854775807, 1) AS try_add(9223372036854775807, 1)#xL] ++- OneRowRelation + + +-- !query +SELECT try_add(-9223372036854775808L, -1) +-- !query analysis +Project [try_add(-9223372036854775808, -1) AS try_add(-9223372036854775808, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT try_add(1, (2147483647 + 1)) +-- !query analysis +Project [try_add(1, (2147483647 + 1)) AS try_add(1, (2147483647 + 1))#x] ++- OneRowRelation + + +-- !query +SELECT try_add(1L, (9223372036854775807L + 1L)) +-- !query analysis +Project [try_add(1, (9223372036854775807 + 1)) AS try_add(1, (9223372036854775807 + 1))#xL] ++- OneRowRelation + + +-- !query +SELECT try_add(1, 1.0 / 0.0) +-- !query analysis +Project [try_add(1, (1.0 / 0.0)) AS try_add(1, (1.0 / 0.0))#x] ++- OneRowRelation + + +-- !query +SELECT try_add(date'2021-01-01', 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(1, date'2021-01-01') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(date'2021-01-01', interval 2 year) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(date'2021-01-01', interval 2 second) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(interval 2 year, date'2021-01-01') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(interval 2 second, date'2021-01-01') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(timestamp_ltz'2021-01-01 00:00:00', interval 2 year) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(timestamp_ntz'2021-01-01 00:00:00', interval 2 second) +-- !query analysis +Project [try_add(2021-01-01 00:00:00, INTERVAL '02' SECOND) AS try_add(TIMESTAMP_NTZ '2021-01-01 00:00:00', INTERVAL '02' SECOND)#x] ++- OneRowRelation + + +-- !query +SELECT try_add(interval 2 year, timestamp_ltz'2021-01-01 00:00:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(interval 2 second, timestamp_ntz'2021-01-01 00:00:00') +-- !query analysis +Project [try_add(INTERVAL '02' SECOND, 2021-01-01 00:00:00) AS try_add(INTERVAL '02' SECOND, TIMESTAMP_NTZ '2021-01-01 00:00:00')#x] ++- OneRowRelation + + +-- !query +SELECT try_add(interval 2 year, interval 2 year) +-- !query analysis +Project [try_add(INTERVAL '2' YEAR, INTERVAL '2' YEAR) AS try_add(INTERVAL '2' YEAR, INTERVAL '2' YEAR)#x] ++- OneRowRelation + + +-- !query +SELECT try_add(interval 2 second, interval 2 second) +-- !query analysis +Project [try_add(INTERVAL '02' SECOND, INTERVAL '02' SECOND) AS try_add(INTERVAL '02' SECOND, INTERVAL '02' SECOND)#x] ++- OneRowRelation + + +-- !query +SELECT try_add(interval 2 year, interval 2 second) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2' YEAR\"", + "inputType" : "\"INTERVAL YEAR\"", + "paramIndex" : "1", + "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 analysis +Project [try_add(INTERVAL '2147483647' MONTH, INTERVAL '2' MONTH) AS try_add(INTERVAL '2147483647' MONTH, INTERVAL '2' MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT try_add(interval 106751991 day, interval 3 day) +-- !query analysis +Project [try_add(INTERVAL '106751991' DAY, INTERVAL '3' DAY) AS try_add(INTERVAL '106751991' DAY, INTERVAL '3' DAY)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(1, 0.5) +-- !query analysis +Project [try_divide(1, 0.5) AS try_divide(1, 0.5)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(1, 0) +-- !query analysis +Project [try_divide(1, 0) AS try_divide(1, 0)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(0, 0) +-- !query analysis +Project [try_divide(0, 0) AS try_divide(0, 0)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(1, (2147483647 + 1)) +-- !query analysis +Project [try_divide(1, (2147483647 + 1)) AS try_divide(1, (2147483647 + 1))#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(1L, (9223372036854775807L + 1L)) +-- !query analysis +Project [try_divide(1, (9223372036854775807 + 1)) AS try_divide(1, (9223372036854775807 + 1))#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(1, 1.0 / 0.0) +-- !query analysis +Project [try_divide(1, (1.0 / 0.0)) AS try_divide(1, (1.0 / 0.0))#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(interval 2 year, 2) +-- !query analysis +Project [try_divide(INTERVAL '2' YEAR, 2) AS try_divide(INTERVAL '2' YEAR, 2)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(interval 2 second, 2) +-- !query analysis +Project [try_divide(INTERVAL '02' SECOND, 2) AS try_divide(INTERVAL '02' SECOND, 2)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(interval 2 year, 0) +-- !query analysis +Project [try_divide(INTERVAL '2' YEAR, 0) AS try_divide(INTERVAL '2' YEAR, 0)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(interval 2 second, 0) +-- !query analysis +Project [try_divide(INTERVAL '02' SECOND, 0) AS try_divide(INTERVAL '02' SECOND, 0)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(interval 2147483647 month, 0.5) +-- !query analysis +Project [try_divide(INTERVAL '2147483647' MONTH, 0.5) AS try_divide(INTERVAL '2147483647' MONTH, 0.5)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(interval 106751991 day, 0.5) +-- !query analysis +Project [try_divide(INTERVAL '106751991' DAY, 0.5) AS try_divide(INTERVAL '106751991' DAY, 0.5)#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(1, 1) +-- !query analysis +Project [try_subtract(1, 1) AS try_subtract(1, 1)#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(2147483647, -1) +-- !query analysis +Project [try_subtract(2147483647, -1) AS try_subtract(2147483647, -1)#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(-2147483648, 1) +-- !query analysis +Project [try_subtract(-2147483648, 1) AS try_subtract(-2147483648, 1)#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(9223372036854775807L, -1) +-- !query analysis +Project [try_subtract(9223372036854775807, -1) AS try_subtract(9223372036854775807, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT try_subtract(-9223372036854775808L, 1) +-- !query analysis +Project [try_subtract(-9223372036854775808, 1) AS try_subtract(-9223372036854775808, 1)#xL] ++- OneRowRelation + + +-- !query +SELECT try_subtract(1, (2147483647 + 1)) +-- !query analysis +Project [try_subtract(1, (2147483647 + 1)) AS try_subtract(1, (2147483647 + 1))#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(1L, (9223372036854775807L + 1L)) +-- !query analysis +Project [try_subtract(1, (9223372036854775807 + 1)) AS try_subtract(1, (9223372036854775807 + 1))#xL] ++- OneRowRelation + + +-- !query +SELECT try_subtract(1, 1.0 / 0.0) +-- !query analysis +Project [try_subtract(1, (1.0 / 0.0)) AS try_subtract(1, (1.0 / 0.0))#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(interval 2 year, interval 3 year) +-- !query analysis +Project [try_subtract(INTERVAL '2' YEAR, INTERVAL '3' YEAR) AS try_subtract(INTERVAL '2' YEAR, INTERVAL '3' YEAR)#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(interval 3 second, interval 2 second) +-- !query analysis +Project [try_subtract(INTERVAL '03' SECOND, INTERVAL '02' SECOND) AS try_subtract(INTERVAL '03' SECOND, INTERVAL '02' SECOND)#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(interval 2147483647 month, interval -2 month) +-- !query analysis +Project [try_subtract(INTERVAL '2147483647' MONTH, INTERVAL '-2' MONTH) AS try_subtract(INTERVAL '2147483647' MONTH, INTERVAL '-2' MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(interval 106751991 day, interval -3 day) +-- !query analysis +Project [try_subtract(INTERVAL '106751991' DAY, INTERVAL '-3' DAY) AS try_subtract(INTERVAL '106751991' DAY, INTERVAL '-3' DAY)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(2, 3) +-- !query analysis +Project [try_multiply(2, 3) AS try_multiply(2, 3)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(2147483647, -2) +-- !query analysis +Project [try_multiply(2147483647, -2) AS try_multiply(2147483647, -2)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(-2147483648, 2) +-- !query analysis +Project [try_multiply(-2147483648, 2) AS try_multiply(-2147483648, 2)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(9223372036854775807L, 2) +-- !query analysis +Project [try_multiply(9223372036854775807, 2) AS try_multiply(9223372036854775807, 2)#xL] ++- OneRowRelation + + +-- !query +SELECT try_multiply(-9223372036854775808L, -2) +-- !query analysis +Project [try_multiply(-9223372036854775808, -2) AS try_multiply(-9223372036854775808, -2)#xL] ++- OneRowRelation + + +-- !query +SELECT try_multiply(1, (2147483647 + 1)) +-- !query analysis +Project [try_multiply(1, (2147483647 + 1)) AS try_multiply(1, (2147483647 + 1))#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(1L, (9223372036854775807L + 1L)) +-- !query analysis +Project [try_multiply(1, (9223372036854775807 + 1)) AS try_multiply(1, (9223372036854775807 + 1))#xL] ++- OneRowRelation + + +-- !query +SELECT try_multiply(1, 1.0 / 0.0) +-- !query analysis +Project [try_multiply(1, (1.0 / 0.0)) AS try_multiply(1, (1.0 / 0.0))#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(interval 2 year, 2) +-- !query analysis +Project [try_multiply(INTERVAL '2' YEAR, 2) AS try_multiply(INTERVAL '2' YEAR, 2)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(interval 2 second, 2) +-- !query analysis +Project [try_multiply(INTERVAL '02' SECOND, 2) AS try_multiply(INTERVAL '02' SECOND, 2)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(interval 2 year, 0) +-- !query analysis +Project [try_multiply(INTERVAL '2' YEAR, 0) AS try_multiply(INTERVAL '2' YEAR, 0)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(interval 2 second, 0) +-- !query analysis +Project [try_multiply(INTERVAL '02' SECOND, 0) AS try_multiply(INTERVAL '02' SECOND, 0)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(interval 2147483647 month, 2) +-- !query analysis +Project [try_multiply(INTERVAL '2147483647' MONTH, 2) AS try_multiply(INTERVAL '2147483647' MONTH, 2)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(interval 106751991 day, 2) +-- !query analysis +Project [try_multiply(INTERVAL '106751991' DAY, 2) AS try_multiply(INTERVAL '106751991' DAY, 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/ansi/try_datetime_functions.sql.out new file mode 100644 index 0000000000000..27751e0503a61 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_datetime_functions.sql.out @@ -0,0 +1,41 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select try_to_timestamp(null), try_to_timestamp('2016-12-31 00:12:00'), try_to_timestamp('2016-12-31', 'yyyy-MM-dd') +-- !query analysis +Project [try_to_timestamp(cast(null as string), None, TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(NULL)#x, try_to_timestamp(2016-12-31 00:12:00, None, TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(2016-12-31 00:12:00)#x, try_to_timestamp(2016-12-31, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(2016-12-31, yyyy-MM-dd)#x] ++- OneRowRelation + + +-- !query +select try_to_timestamp(1) +-- !query analysis +Project [try_to_timestamp(1, None, TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(1)#x] ++- OneRowRelation + + +-- !query +select try_to_timestamp('2016-12-31 abc') +-- !query analysis +Project [try_to_timestamp(2016-12-31 abc, None, TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(2016-12-31 abc)#x] ++- OneRowRelation + + +-- !query +select try_to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query analysis +Project [try_to_timestamp(2019-10-06 10:11:12., Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(2019-10-06 10:11:12., yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] ++- OneRowRelation + + +-- !query +select try_to_timestamp("02-29", "MM-dd") +-- !query analysis +Project [try_to_timestamp(02-29, Some(MM-dd), TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(02-29, MM-dd)#x] ++- OneRowRelation + + +-- !query +select try_to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') +-- !query analysis +Project [try_to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEEE), TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEEE)#x] ++- OneRowRelation 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/ansi/try_element_at.sql.out new file mode 100644 index 0000000000000..2475e315884f7 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_element_at.sql.out @@ -0,0 +1,55 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_element_at(array(1, 2, 3), 0) +-- !query analysis +Project [try_element_at(array(1, 2, 3), 0) AS try_element_at(array(1, 2, 3), 0)#x] ++- OneRowRelation + + +-- !query +SELECT try_element_at(array(1, 2, 3), 1) +-- !query analysis +Project [try_element_at(array(1, 2, 3), 1) AS try_element_at(array(1, 2, 3), 1)#x] ++- OneRowRelation + + +-- !query +SELECT try_element_at(array(1, 2, 3), 3) +-- !query analysis +Project [try_element_at(array(1, 2, 3), 3) AS try_element_at(array(1, 2, 3), 3)#x] ++- OneRowRelation + + +-- !query +SELECT try_element_at(array(1, 2, 3), 4) +-- !query analysis +Project [try_element_at(array(1, 2, 3), 4) AS try_element_at(array(1, 2, 3), 4)#x] ++- OneRowRelation + + +-- !query +SELECT try_element_at(array(1, 2, 3), -1) +-- !query analysis +Project [try_element_at(array(1, 2, 3), -1) AS try_element_at(array(1, 2, 3), -1)#x] ++- OneRowRelation + + +-- !query +SELECT try_element_at(array(1, 2, 3), -4) +-- !query analysis +Project [try_element_at(array(1, 2, 3), -4) AS try_element_at(array(1, 2, 3), -4)#x] ++- OneRowRelation + + +-- !query +SELECT try_element_at(map('a','b'), 'a') +-- !query analysis +Project [try_element_at(map(a, b), a) AS try_element_at(map(a, b), a)#x] ++- OneRowRelation + + +-- !query +SELECT try_element_at(map('a','b'), 'abc') +-- !query analysis +Project [try_element_at(map(a, b), abc) AS try_element_at(map(a, b), abc)#x] ++- OneRowRelation 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 a9c18ab869025..91cbd844d998c 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 @@ -129,13 +129,7 @@ select array_contains(timestamp_array, timestamp '2016-11-15 20:54:00.000'), array_contains(timestamp_array, timestamp '2016-01-01 20:54:00.000') from primitive_arrays -- !query analysis -Project [array_contains(boolean_array#x, true) AS array_contains(boolean_array, true)#x, array_contains(boolean_array#x, false) AS array_contains(boolean_array, false)#x, array_contains(tinyint_array#x, 2) AS array_contains(tinyint_array, 2)#x, array_contains(tinyint_array#x, 0) AS array_contains(tinyint_array, 0)#x, array_contains(smallint_array#x, 2) AS array_contains(smallint_array, 2)#x, array_contains(smallint_array#x, 0) AS array_contains(smallint_array, 0)#x, array_contains(int_array#x, 2) AS array_contains(int_array, 2)#x, array_contains(int_array#x, 0) AS array_contains(int_array, 0)#x, array_contains(bigint_array#x, 2) AS array_contains(bigint_array, 2)#x, array_contains(bigint_array#x, 0) AS array_contains(bigint_array, 0)#x, array_contains(decimal_array#x, 9223372036854775809) AS array_contains(decimal_array, 9223372036854775809)#x, array_contains(decimal_array#x, cast(1 as decimal(19,0))) AS array_contains(decimal_array, 1)#x, array_contains(double_array#x, 2.0) AS array_contains(double_array, 2.0)#x, array_contains(double_array#x, 0.0) AS array_contains(double_array, 0.0)#x, array_contains(float_array#x, cast(2.0 as float)) AS array_contains(float_array, 2.0)#x, array_contains(float_array#x, cast(0.0 as float)) AS array_contains(float_array, 0.0)#x, array_contains(date_array#x, 2016-03-14) AS array_contains(date_array, DATE '2016-03-14')#x, array_contains(date_array#x, 2016-01-01) AS array_contains(date_array, DATE '2016-01-01')#x, array_contains(timestamp_array#x, 2016-11-15 20:54:00) AS array_contains(timestamp_array, TIMESTAMP '2016-11-15 20:54:00')#x, array_contains(timestamp_array#x, 2016-01-01 20:54:00) AS array_contains(timestamp_array, TIMESTAMP '2016-01-01 20:54:00')#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] - +- Project [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] - +- SubqueryAlias primitive_arrays - +- LocalRelation [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] +[Analyzer test output redacted due to nondeterminism] -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/bitwise.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/bitwise.sql.out new file mode 100644 index 0000000000000..ca960fc1aca4b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/bitwise.sql.out @@ -0,0 +1,308 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select bit_count(null) +-- !query analysis +Project [bit_count(null) AS bit_count(NULL)#x] ++- OneRowRelation + + +-- !query +select bit_count(true) +-- !query analysis +Project [bit_count(true) AS bit_count(true)#x] ++- OneRowRelation + + +-- !query +select bit_count(false) +-- !query analysis +Project [bit_count(false) AS bit_count(false)#x] ++- OneRowRelation + + +-- !query +select bit_count(cast(1 as tinyint)) +-- !query analysis +Project [bit_count(cast(1 as tinyint)) AS bit_count(CAST(1 AS TINYINT))#x] ++- OneRowRelation + + +-- !query +select bit_count(cast(2 as tinyint)) +-- !query analysis +Project [bit_count(cast(2 as tinyint)) AS bit_count(CAST(2 AS TINYINT))#x] ++- OneRowRelation + + +-- !query +select bit_count(cast(3 as tinyint)) +-- !query analysis +Project [bit_count(cast(3 as tinyint)) AS bit_count(CAST(3 AS TINYINT))#x] ++- OneRowRelation + + +-- !query +select bit_count(1S) +-- !query analysis +Project [bit_count(1) AS bit_count(1)#x] ++- OneRowRelation + + +-- !query +select bit_count(2S) +-- !query analysis +Project [bit_count(2) AS bit_count(2)#x] ++- OneRowRelation + + +-- !query +select bit_count(3S) +-- !query analysis +Project [bit_count(3) AS bit_count(3)#x] ++- OneRowRelation + + +-- !query +select bit_count(1) +-- !query analysis +Project [bit_count(1) AS bit_count(1)#x] ++- OneRowRelation + + +-- !query +select bit_count(2) +-- !query analysis +Project [bit_count(2) AS bit_count(2)#x] ++- OneRowRelation + + +-- !query +select bit_count(3) +-- !query analysis +Project [bit_count(3) AS bit_count(3)#x] ++- OneRowRelation + + +-- !query +select bit_count(1L) +-- !query analysis +Project [bit_count(1) AS bit_count(1)#x] ++- OneRowRelation + + +-- !query +select bit_count(2L) +-- !query analysis +Project [bit_count(2) AS bit_count(2)#x] ++- OneRowRelation + + +-- !query +select bit_count(3L) +-- !query analysis +Project [bit_count(3) AS bit_count(3)#x] ++- OneRowRelation + + +-- !query +select bit_count(-1L) +-- !query analysis +Project [bit_count(-1) AS bit_count(-1)#x] ++- OneRowRelation + + +-- !query +select bit_count(9223372036854775807L) +-- !query analysis +Project [bit_count(9223372036854775807) AS bit_count(9223372036854775807)#x] ++- OneRowRelation + + +-- !query +select bit_count(-9223372036854775808L) +-- !query analysis +Project [bit_count(-9223372036854775808) AS bit_count(-9223372036854775808)#x] ++- OneRowRelation + + +-- !query +select bit_count("bit count") +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"bit count\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "(\"INTEGRAL\" or \"BOOLEAN\")", + "sqlExpr" : "\"bit_count(bit count)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "bit_count(\"bit count\")" + } ] +} + + +-- !query +select bit_count('a') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"a\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "(\"INTEGRAL\" or \"BOOLEAN\")", + "sqlExpr" : "\"bit_count(a)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "bit_count('a')" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (2, 3, 4, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4) +-- !query analysis +CreateViewCommand `bitwise_test`, SELECT * FROM VALUES + (1, 1, 1, 1L), + (2, 3, 4, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4), false, true, LocalTempView, true + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT BIT_XOR(b3) AS n1 FROM bitwise_test where 1 = 0 +-- !query analysis +Aggregate [bit_xor(b3#x) AS n1#x] ++- Filter (1 = 0) + +- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT BIT_XOR(b4) AS n1 FROM bitwise_test where b4 is null +-- !query analysis +Aggregate [bit_xor(b4#xL) AS n1#xL] ++- Filter isnull(b4#xL) + +- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT + BIT_XOR(cast(b1 as tinyint)) AS a4, + BIT_XOR(cast(b2 as smallint)) AS b5, + BIT_XOR(b3) AS c2, + BIT_XOR(b4) AS d2, + BIT_XOR(distinct b4) AS e2 +FROM bitwise_test +-- !query analysis +Aggregate [bit_xor(cast(b1#x as tinyint)) AS a4#x, bit_xor(cast(b2#x as smallint)) AS b5#x, bit_xor(b3#x) AS c2#x, bit_xor(b4#xL) AS d2#xL, bit_xor(distinct b4#xL) AS e2#xL] ++- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT bit_xor(b3) FROM bitwise_test GROUP BY b1 & 1 +-- !query analysis +Aggregate [(b1#x & 1)], [bit_xor(b3#x) AS bit_xor(b3)#x] ++- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT b1, bit_xor(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7 +-- !query analysis +Project [b1#x, bit_xor(b2)#x] ++- Filter (bit_and(b2#x)#x < 7) + +- Aggregate [b1#x], [b1#x, bit_xor(b2#x) AS bit_xor(b2)#x, bit_and(b2#x) AS bit_and(b2#x)#x] + +- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT b1, b2, bit_xor(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test +-- !query analysis +Project [b1#x, b2#x, bit_xor(b2) OVER (PARTITION BY b1 ORDER BY b2 ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [b1#x, b2#x, bit_xor(b2) OVER (PARTITION BY b1 ORDER BY b2 ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, bit_xor(b2) OVER (PARTITION BY b1 ORDER BY b2 ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [bit_xor(b2#x) windowspecdefinition(b1#x, b2#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS bit_xor(b2) OVER (PARTITION BY b1 ORDER BY b2 ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [b1#x], [b2#x ASC NULLS FIRST] + +- Project [b1#x, b2#x] + +- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +select getbit(11L, 3), getbit(11L, 2), getbit(11L, 1), getbit(11L, 0) +-- !query analysis +Project [getbit(11, 3) AS getbit(11, 3)#x, getbit(11, 2) AS getbit(11, 2)#x, getbit(11, 1) AS getbit(11, 1)#x, getbit(11, 0) AS getbit(11, 0)#x] ++- OneRowRelation + + +-- !query +select getbit(11L, 2 + 1), getbit(11L, 3 - 1), getbit(10L + 1, 1 * 1), getbit(cast(11L / 1 AS long), 1 - 1) +-- !query analysis +Project [getbit(11, (2 + 1)) AS getbit(11, (2 + 1))#x, getbit(11, (3 - 1)) AS getbit(11, (3 - 1))#x, getbit((10 + cast(1 as bigint)), (1 * 1)) AS getbit((10 + 1), (1 * 1))#x, getbit(cast((cast(11 as double) / cast(1 as double)) as bigint), (1 - 1)) AS getbit(CAST((11 / 1) AS BIGINT), (1 - 1))#x] ++- OneRowRelation + + +-- !query +select getbit(11L, 63) +-- !query analysis +Project [getbit(11, 63) AS getbit(11, 63)#x] ++- OneRowRelation + + +-- !query +select getbit(11L, -1) +-- !query analysis +Project [getbit(11, -1) AS getbit(11, -1)#x] ++- OneRowRelation + + +-- !query +select getbit(11L, 64) +-- !query analysis +Project [getbit(11, 64) AS getbit(11, 64)#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 new file mode 100644 index 0000000000000..ad9b63e0fed14 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cast.sql.out @@ -0,0 +1,745 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT CAST('1.23' AS int) +-- !query analysis +Project [cast(1.23 as int) AS CAST(1.23 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('1.23' AS long) +-- !query analysis +Project [cast(1.23 as bigint) AS CAST(1.23 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('-4.56' AS int) +-- !query analysis +Project [cast(-4.56 as int) AS CAST(-4.56 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('-4.56' AS long) +-- !query analysis +Project [cast(-4.56 as bigint) AS CAST(-4.56 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('abc' AS int) +-- !query analysis +Project [cast(abc as int) AS CAST(abc AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('abc' AS long) +-- !query analysis +Project [cast(abc as bigint) AS CAST(abc AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('abc' AS float) +-- !query analysis +Project [cast(abc as float) AS CAST(abc AS FLOAT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('abc' AS double) +-- !query analysis +Project [cast(abc as double) AS CAST(abc AS DOUBLE)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('1234567890123' AS int) +-- !query analysis +Project [cast(1234567890123 as int) AS CAST(1234567890123 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('12345678901234567890123' AS long) +-- !query analysis +Project [cast(12345678901234567890123 as bigint) AS CAST(12345678901234567890123 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('' AS int) +-- !query analysis +Project [cast( as int) AS CAST( AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('' AS long) +-- !query analysis +Project [cast( as bigint) AS CAST( AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('' AS float) +-- !query analysis +Project [cast( as float) AS CAST( AS FLOAT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('' AS double) +-- !query analysis +Project [cast( as double) AS CAST( AS DOUBLE)#x] ++- OneRowRelation + + +-- !query +SELECT CAST(NULL AS int) +-- !query analysis +Project [cast(null as int) AS CAST(NULL AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST(NULL AS long) +-- !query analysis +Project [cast(null as bigint) AS CAST(NULL AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('123.a' AS int) +-- !query analysis +Project [cast(123.a as int) AS CAST(123.a AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('123.a' AS long) +-- !query analysis +Project [cast(123.a as bigint) AS CAST(123.a AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('123.a' AS float) +-- !query analysis +Project [cast(123.a as float) AS CAST(123.a AS FLOAT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('123.a' AS double) +-- !query analysis +Project [cast(123.a as double) AS CAST(123.a AS DOUBLE)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('-2147483648' AS int) +-- !query analysis +Project [cast(-2147483648 as int) AS CAST(-2147483648 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('-2147483649' AS int) +-- !query analysis +Project [cast(-2147483649 as int) AS CAST(-2147483649 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('2147483647' AS int) +-- !query analysis +Project [cast(2147483647 as int) AS CAST(2147483647 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('2147483648' AS int) +-- !query analysis +Project [cast(2147483648 as int) AS CAST(2147483648 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT CAST('-9223372036854775808' AS long) +-- !query analysis +Project [cast(-9223372036854775808 as bigint) AS CAST(-9223372036854775808 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('-9223372036854775809' AS long) +-- !query analysis +Project [cast(-9223372036854775809 as bigint) AS CAST(-9223372036854775809 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('9223372036854775807' AS long) +-- !query analysis +Project [cast(9223372036854775807 as bigint) AS CAST(9223372036854775807 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST('9223372036854775808' AS long) +-- !query analysis +Project [cast(9223372036854775808 as bigint) AS CAST(9223372036854775808 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT HEX(CAST('abc' AS binary)) +-- !query analysis +Project [hex(cast(abc as binary)) AS hex(CAST(abc AS BINARY))#x] ++- OneRowRelation + + +-- !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 + + +-- !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 + + +-- !query +SELECT HEX(CAST(123S AS binary)) +-- !query analysis +Project [hex(cast(123 as binary)) AS hex(CAST(123 AS BINARY))#x] ++- OneRowRelation + + +-- !query +SELECT HEX(CAST(-123S AS binary)) +-- !query analysis +Project [hex(cast(-123 as binary)) AS hex(CAST(-123 AS BINARY))#x] ++- OneRowRelation + + +-- !query +SELECT HEX(CAST(123 AS binary)) +-- !query analysis +Project [hex(cast(123 as binary)) AS hex(CAST(123 AS BINARY))#x] ++- OneRowRelation + + +-- !query +SELECT HEX(CAST(-123 AS binary)) +-- !query analysis +Project [hex(cast(-123 as binary)) AS hex(CAST(-123 AS BINARY))#x] ++- OneRowRelation + + +-- !query +SELECT HEX(CAST(123L AS binary)) +-- !query analysis +Project [hex(cast(123 as binary)) AS hex(CAST(123 AS BINARY))#x] ++- OneRowRelation + + +-- !query +SELECT HEX(CAST(-123L AS binary)) +-- !query analysis +Project [hex(cast(-123 as binary)) AS hex(CAST(-123 AS BINARY))#x] ++- OneRowRelation + + +-- !query +DESC FUNCTION boolean +-- !query analysis +DescribeFunctionCommand org.apache.spark.sql.catalyst.expressions.ExpressionInfo@xxxxxxxx, false + + +-- !query +DESC FUNCTION EXTENDED boolean +-- !query analysis +DescribeFunctionCommand org.apache.spark.sql.catalyst.expressions.ExpressionInfo@xxxxxxxx, true + + +-- !query +SELECT CAST('interval 3 month 1 hour' AS interval) +-- !query analysis +Project [cast(interval 3 month 1 hour as interval) AS CAST(interval 3 month 1 hour AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +SELECT CAST("interval '3-1' year to month" AS interval year to month) +-- !query analysis +Project [cast(interval '3-1' year to month as interval year to month) AS CAST(interval '3-1' year to month AS INTERVAL YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT CAST("interval '3 00:00:01' day to second" AS interval day to second) +-- !query analysis +Project [cast(interval '3 00:00:01' day to second as interval day to second) AS CAST(interval '3 00:00:01' day to second AS INTERVAL DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +SELECT CAST(interval 3 month 1 hour AS string) +-- !query analysis +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 analysis +Project [cast(INTERVAL '3-1' YEAR TO MONTH as string) AS CAST(INTERVAL '3-1' YEAR TO MONTH AS STRING)#x] ++- OneRowRelation + + +-- !query +SELECT CAST(interval 3 day 1 second AS string) +-- !query analysis +Project [cast(INTERVAL '3 00:00:01' DAY TO SECOND as string) AS CAST(INTERVAL '3 00:00:01' DAY TO SECOND AS STRING)#x] ++- OneRowRelation + + +-- !query +select cast(' 1' as tinyint) +-- !query analysis +Project [cast( 1 as tinyint) AS CAST( 1 AS TINYINT)#x] ++- OneRowRelation + + +-- !query +select cast(' 1\t' as tinyint) +-- !query analysis +Project [cast( 1 as tinyint) AS CAST( 1 AS TINYINT)#x] ++- OneRowRelation + + +-- !query +select cast(' 1' as smallint) +-- !query analysis +Project [cast( 1 as smallint) AS CAST( 1 AS SMALLINT)#x] ++- OneRowRelation + + +-- !query +select cast(' 1' as INT) +-- !query analysis +Project [cast( 1 as int) AS CAST( 1 AS INT)#x] ++- OneRowRelation + + +-- !query +select cast(' 1' as bigint) +-- !query analysis +Project [cast( 1 as bigint) AS CAST( 1 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +select cast(' 1' as float) +-- !query analysis +Project [cast( 1 as float) AS CAST( 1 AS FLOAT)#x] ++- OneRowRelation + + +-- !query +select cast(' 1 ' as DOUBLE) +-- !query analysis +Project [cast( 1 as double) AS CAST( 1 AS DOUBLE)#x] ++- OneRowRelation + + +-- !query +select cast('1.0 ' as DEC) +-- !query analysis +Project [cast(1.0 as decimal(10,0)) AS CAST(1.0 AS DECIMAL(10,0))#x] ++- OneRowRelation + + +-- !query +select cast('1中文' as tinyint) +-- !query analysis +Project [cast(1中文 as tinyint) AS CAST(1中文 AS TINYINT)#x] ++- OneRowRelation + + +-- !query +select cast('1中文' as smallint) +-- !query analysis +Project [cast(1中文 as smallint) AS CAST(1中文 AS SMALLINT)#x] ++- OneRowRelation + + +-- !query +select cast('1中文' as INT) +-- !query analysis +Project [cast(1中文 as int) AS CAST(1中文 AS INT)#x] ++- OneRowRelation + + +-- !query +select cast('中文1' as bigint) +-- !query analysis +Project [cast(中文1 as bigint) AS CAST(中文1 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +select cast('1中文' as bigint) +-- !query analysis +Project [cast(1中文 as bigint) AS CAST(1中文 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +select cast('\t\t true \n\r ' as boolean) +-- !query analysis +Project [cast( true + as boolean) AS CAST( true + AS BOOLEAN)#x] ++- OneRowRelation + + +-- !query +select cast('\t\n false \t\r' as boolean) +-- !query analysis +Project [cast( + false as boolean) AS CAST( + false AS BOOLEAN)#x] ++- OneRowRelation + + +-- !query +select cast('\t\n xyz \t\r' as boolean) +-- !query analysis +Project [cast( + xyz as boolean) AS CAST( + xyz AS BOOLEAN)#x] ++- OneRowRelation + + +-- !query +select cast('23.45' as decimal(4, 2)) +-- !query analysis +Project [cast(23.45 as decimal(4,2)) AS CAST(23.45 AS DECIMAL(4,2))#x] ++- OneRowRelation + + +-- !query +select cast('123.45' as decimal(4, 2)) +-- !query analysis +Project [cast(123.45 as decimal(4,2)) AS CAST(123.45 AS DECIMAL(4,2))#x] ++- OneRowRelation + + +-- !query +select cast('xyz' as decimal(4, 2)) +-- !query analysis +Project [cast(xyz as decimal(4,2)) AS CAST(xyz AS DECIMAL(4,2))#x] ++- OneRowRelation + + +-- !query +select cast('2022-01-01' as date) +-- !query analysis +Project [cast(2022-01-01 as date) AS CAST(2022-01-01 AS DATE)#x] ++- OneRowRelation + + +-- !query +select cast('a' as date) +-- !query analysis +Project [cast(a as date) AS CAST(a AS DATE)#x] ++- OneRowRelation + + +-- !query +select cast('2022-01-01 00:00:00' as timestamp) +-- !query analysis +Project [cast(2022-01-01 00:00:00 as timestamp) AS CAST(2022-01-01 00:00:00 AS TIMESTAMP)#x] ++- OneRowRelation + + +-- !query +select cast('a' as timestamp) +-- !query analysis +Project [cast(a as timestamp) AS CAST(a AS TIMESTAMP)#x] ++- OneRowRelation + + +-- !query +select cast('2022-01-01 00:00:00' as timestamp_ntz) +-- !query analysis +Project [cast(2022-01-01 00:00:00 as timestamp_ntz) AS CAST(2022-01-01 00:00:00 AS TIMESTAMP_NTZ)#x] ++- OneRowRelation + + +-- !query +select cast('a' as timestamp_ntz) +-- !query analysis +Project [cast(a as timestamp_ntz) AS CAST(a AS TIMESTAMP_NTZ)#x] ++- OneRowRelation + + +-- !query +select cast(cast('inf' as double) as timestamp) +-- !query analysis +Project [cast(cast(inf as double) as timestamp) AS CAST(CAST(inf AS DOUBLE) AS TIMESTAMP)#x] ++- OneRowRelation + + +-- !query +select cast(cast('inf' as float) as timestamp) +-- !query analysis +Project [cast(cast(inf as float) as timestamp) AS CAST(CAST(inf AS FLOAT) AS TIMESTAMP)#x] ++- OneRowRelation + + +-- !query +select cast(interval '1' year as tinyint) +-- !query analysis +Project [cast(INTERVAL '1' YEAR as tinyint) AS CAST(INTERVAL '1' YEAR AS TINYINT)#x] ++- OneRowRelation + + +-- !query +select cast(interval '-10-2' year to month as smallint) +-- !query analysis +Project [cast(INTERVAL '-10-2' YEAR TO MONTH as smallint) AS CAST(INTERVAL '-10-2' YEAR TO MONTH AS SMALLINT)#x] ++- OneRowRelation + + +-- !query +select cast(interval '1000' month as int) +-- !query analysis +Project [cast(INTERVAL '1000' MONTH as int) AS CAST(INTERVAL '1000' MONTH AS INT)#x] ++- OneRowRelation + + +-- !query +select cast(interval -'10.123456' second as tinyint) +-- !query analysis +Project [cast(INTERVAL '-10.123456' SECOND as tinyint) AS CAST(INTERVAL '-10.123456' SECOND AS TINYINT)#x] ++- OneRowRelation + + +-- !query +select cast(interval '23:59:59' hour to second as smallint) +-- !query analysis +Project [cast(INTERVAL '23:59:59' HOUR TO SECOND as smallint) AS CAST(INTERVAL '23:59:59' HOUR TO SECOND AS SMALLINT)#x] ++- OneRowRelation + + +-- !query +select cast(interval -'1 02:03:04.123' day to second as int) +-- !query analysis +Project [cast(INTERVAL '-1 02:03:04.123' DAY TO SECOND as int) AS CAST(INTERVAL '-1 02:03:04.123' DAY TO SECOND AS INT)#x] ++- OneRowRelation + + +-- !query +select cast(interval '10' day as bigint) +-- !query analysis +Project [cast(INTERVAL '10' DAY as bigint) AS CAST(INTERVAL '10' DAY AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +select cast(interval '-1000' month as tinyint) +-- !query analysis +Project [cast(INTERVAL '-1000' MONTH as tinyint) AS CAST(INTERVAL '-1000' MONTH AS TINYINT)#x] ++- OneRowRelation + + +-- !query +select cast(interval '1000000' second as smallint) +-- !query analysis +Project [cast(INTERVAL '1000000' SECOND as smallint) AS CAST(INTERVAL '1000000' SECOND AS SMALLINT)#x] ++- OneRowRelation + + +-- !query +select cast(1Y as interval year) +-- !query analysis +Project [cast(1 as interval year) AS CAST(1 AS INTERVAL YEAR)#x] ++- OneRowRelation + + +-- !query +select cast(-122S as interval year to month) +-- !query analysis +Project [cast(-122 as interval year to month) AS CAST(-122 AS INTERVAL YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select cast(ym as interval year to month) from values(-122S) as t(ym) +-- !query analysis +Project [cast(ym#x as interval year to month) AS ym#x] ++- SubqueryAlias t + +- LocalRelation [ym#x] + + +-- !query +select cast(1000 as interval month) +-- !query analysis +Project [cast(1000 as interval month) AS CAST(1000 AS INTERVAL MONTH)#x] ++- OneRowRelation + + +-- !query +select cast(-10L as interval second) +-- !query analysis +Project [cast(-10 as interval second) AS CAST(-10 AS INTERVAL SECOND)#x] ++- OneRowRelation + + +-- !query +select cast(100Y as interval hour to second) +-- !query analysis +Project [cast(100 as interval hour to second) AS CAST(100 AS INTERVAL HOUR TO SECOND)#x] ++- OneRowRelation + + +-- !query +select cast(dt as interval hour to second) from values(100Y) as t(dt) +-- !query analysis +Project [cast(dt#x as interval hour to second) AS dt#x] ++- SubqueryAlias t + +- LocalRelation [dt#x] + + +-- !query +select cast(-1000S as interval day to second) +-- !query analysis +Project [cast(-1000 as interval day to second) AS CAST(-1000 AS INTERVAL DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select cast(10 as interval day) +-- !query analysis +Project [cast(10 as interval day) AS CAST(10 AS INTERVAL DAY)#x] ++- OneRowRelation + + +-- !query +select cast(2147483647 as interval year) +-- !query analysis +Project [cast(2147483647 as interval year) AS CAST(2147483647 AS INTERVAL YEAR)#x] ++- OneRowRelation + + +-- !query +select cast(-9223372036854775808L as interval day) +-- !query analysis +Project [cast(-9223372036854775808 as interval day) AS CAST(-9223372036854775808 AS INTERVAL DAY)#x] ++- OneRowRelation + + +-- !query +select cast(interval '-1' year as decimal(10, 0)) +-- !query analysis +Project [cast(INTERVAL '-1' YEAR as decimal(10,0)) AS CAST(INTERVAL '-1' YEAR AS DECIMAL(10,0))#x] ++- OneRowRelation + + +-- !query +select cast(interval '1.000001' second as decimal(10, 6)) +-- !query analysis +Project [cast(INTERVAL '01.000001' SECOND as decimal(10,6)) AS CAST(INTERVAL '01.000001' SECOND AS DECIMAL(10,6))#x] ++- OneRowRelation + + +-- !query +select cast(interval '08:11:10.001' hour to second as decimal(10, 4)) +-- !query analysis +Project [cast(INTERVAL '08:11:10.001' HOUR TO SECOND as decimal(10,4)) AS CAST(INTERVAL '08:11:10.001' HOUR TO SECOND AS DECIMAL(10,4))#x] ++- OneRowRelation + + +-- !query +select cast(interval '1 01:02:03.1' day to second as decimal(8, 1)) +-- !query analysis +Project [cast(INTERVAL '1 01:02:03.1' DAY TO SECOND as decimal(8,1)) AS CAST(INTERVAL '1 01:02:03.1' DAY TO SECOND AS DECIMAL(8,1))#x] ++- OneRowRelation + + +-- !query +select cast(interval '10.123' second as decimal(4, 2)) +-- !query analysis +Project [cast(INTERVAL '10.123' SECOND as decimal(4,2)) AS CAST(INTERVAL '10.123' SECOND AS DECIMAL(4,2))#x] ++- OneRowRelation + + +-- !query +select cast(interval '10.005' second as decimal(4, 2)) +-- !query analysis +Project [cast(INTERVAL '10.005' SECOND as decimal(4,2)) AS CAST(INTERVAL '10.005' SECOND AS DECIMAL(4,2))#x] ++- OneRowRelation + + +-- !query +select cast(interval '10.123' second as decimal(5, 2)) +-- !query analysis +Project [cast(INTERVAL '10.123' SECOND as decimal(5,2)) AS CAST(INTERVAL '10.123' SECOND AS DECIMAL(5,2))#x] ++- OneRowRelation + + +-- !query +select cast(interval '10.123' second as decimal(1, 0)) +-- !query analysis +Project [cast(INTERVAL '10.123' SECOND as decimal(1,0)) AS CAST(INTERVAL '10.123' SECOND AS DECIMAL(1,0))#x] ++- OneRowRelation + + +-- !query +select cast(10.123456BD as interval day to second) +-- !query analysis +Project [cast(10.123456 as interval day to second) AS CAST(10.123456 AS INTERVAL DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select cast(80.654321BD as interval hour to minute) +-- !query analysis +Project [cast(80.654321 as interval hour to minute) AS CAST(80.654321 AS INTERVAL HOUR TO MINUTE)#x] ++- OneRowRelation + + +-- !query +select cast(-10.123456BD as interval year to month) +-- !query analysis +Project [cast(-10.123456 as interval year to month) AS CAST(-10.123456 AS INTERVAL YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select cast(10.654321BD as interval month) +-- !query analysis +Project [cast(10.654321 as interval month) AS CAST(10.654321 AS INTERVAL MONTH)#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ceil-floor-with-scale-param.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ceil-floor-with-scale-param.sql.out new file mode 100644 index 0000000000000..c76b2e5284a42 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ceil-floor-with-scale-param.sql.out @@ -0,0 +1,283 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT CEIL(2.5, 0) +-- !query analysis +Project [ceil(2.5, 0) AS ceil(2.5, 0)#x] ++- OneRowRelation + + +-- !query +SELECT CEIL(3.5, 0) +-- !query analysis +Project [ceil(3.5, 0) AS ceil(3.5, 0)#x] ++- OneRowRelation + + +-- !query +SELECT CEIL(-2.5, 0) +-- !query analysis +Project [ceil(-2.5, 0) AS ceil(-2.5, 0)#x] ++- OneRowRelation + + +-- !query +SELECT CEIL(-3.5, 0) +-- !query analysis +Project [ceil(-3.5, 0) AS ceil(-3.5, 0)#x] ++- OneRowRelation + + +-- !query +SELECT CEIL(-0.35, 1) +-- !query analysis +Project [ceil(-0.35, 1) AS ceil(-0.35, 1)#x] ++- OneRowRelation + + +-- !query +SELECT CEIL(-35, -1) +-- !query analysis +Project [ceil(cast(-35 as decimal(10,0)), -1) AS ceil(-35, -1)#x] ++- OneRowRelation + + +-- !query +SELECT CEIL(-0.1, 0) +-- !query analysis +Project [ceil(-0.1, 0) AS ceil(-0.1, 0)#x] ++- OneRowRelation + + +-- !query +SELECT CEIL(5, 0) +-- !query analysis +Project [ceil(cast(5 as decimal(10,0)), 0) AS ceil(5, 0)#x] ++- OneRowRelation + + +-- !query +SELECT CEIL(3.14115, -3) +-- !query analysis +Project [ceil(3.14115, -3) AS ceil(3.14115, -3)#x] ++- OneRowRelation + + +-- !query +SELECT CEIL(9.9, 0) +-- !query analysis +Project [ceil(9.9, 0) AS ceil(9.9, 0)#x] ++- OneRowRelation + + +-- !query +SELECT CEIL(CAST(99 AS DECIMAL(2, 0)), -1) +-- !query analysis +Project [ceil(cast(99 as decimal(2,0)), -1) AS ceil(CAST(99 AS DECIMAL(2,0)), -1)#x] ++- OneRowRelation + + +-- !query +SELECT CEIL(2.5, null) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1100", + "messageParameters" : { + "argName" : "scale", + "funcName" : "ceil", + "requiredType" : "int" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "CEIL(2.5, null)" + } ] +} + + +-- !query +SELECT CEIL(2.5, 'a') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1100", + "messageParameters" : { + "argName" : "scale", + "funcName" : "ceil", + "requiredType" : "int" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "CEIL(2.5, 'a')" + } ] +} + + +-- !query +SELECT CEIL(2.5, 0, 0) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "2", + "functionName" : "`ceil`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "CEIL(2.5, 0, 0)" + } ] +} + + +-- !query +SELECT FLOOR(2.5, 0) +-- !query analysis +Project [floor(2.5, 0) AS floor(2.5, 0)#x] ++- OneRowRelation + + +-- !query +SELECT FLOOR(3.5, 0) +-- !query analysis +Project [floor(3.5, 0) AS floor(3.5, 0)#x] ++- OneRowRelation + + +-- !query +SELECT FLOOR(-2.5, 0) +-- !query analysis +Project [floor(-2.5, 0) AS floor(-2.5, 0)#x] ++- OneRowRelation + + +-- !query +SELECT FLOOR(-3.5, 0) +-- !query analysis +Project [floor(-3.5, 0) AS floor(-3.5, 0)#x] ++- OneRowRelation + + +-- !query +SELECT FLOOR(-0.35, 1) +-- !query analysis +Project [floor(-0.35, 1) AS floor(-0.35, 1)#x] ++- OneRowRelation + + +-- !query +SELECT FLOOR(-35, -1) +-- !query analysis +Project [floor(cast(-35 as decimal(10,0)), -1) AS floor(-35, -1)#x] ++- OneRowRelation + + +-- !query +SELECT FLOOR(-0.1, 0) +-- !query analysis +Project [floor(-0.1, 0) AS floor(-0.1, 0)#x] ++- OneRowRelation + + +-- !query +SELECT FLOOR(5, 0) +-- !query analysis +Project [floor(cast(5 as decimal(10,0)), 0) AS floor(5, 0)#x] ++- OneRowRelation + + +-- !query +SELECT FLOOR(3.14115, -3) +-- !query analysis +Project [floor(3.14115, -3) AS floor(3.14115, -3)#x] ++- OneRowRelation + + +-- !query +SELECT FLOOR(-9.9, 0) +-- !query analysis +Project [floor(-9.9, 0) AS floor(-9.9, 0)#x] ++- OneRowRelation + + +-- !query +SELECT FLOOR(CAST(-99 AS DECIMAL(2, 0)), -1) +-- !query analysis +Project [floor(cast(-99 as decimal(2,0)), -1) AS floor(CAST(-99 AS DECIMAL(2,0)), -1)#x] ++- OneRowRelation + + +-- !query +SELECT FLOOR(2.5, null) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1100", + "messageParameters" : { + "argName" : "scale", + "funcName" : "floor", + "requiredType" : "int" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "FLOOR(2.5, null)" + } ] +} + + +-- !query +SELECT FLOOR(2.5, 'a') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1100", + "messageParameters" : { + "argName" : "scale", + "funcName" : "floor", + "requiredType" : "int" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "FLOOR(2.5, 'a')" + } ] +} + + +-- !query +SELECT FLOOR(2.5, 0, 0) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "2", + "functionName" : "`floor`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "FLOOR(2.5, 0, 0)" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/change-column.sql.out new file mode 100644 index 0000000000000..a996aaef4ca3f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/change-column.sql.out @@ -0,0 +1,264 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE test_change(a INT, b STRING, c INT) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`test_change`, false + + +-- !query +DESC test_change +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`test_change`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE test_change CHANGE a +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0035", + "messageParameters" : { + "message" : "ALTER TABLE table CHANGE COLUMN requires a TYPE, a SET/DROP, a COMMENT, or a FIRST/AFTER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 32, + "fragment" : "ALTER TABLE test_change CHANGE a" + } ] +} + + +-- !query +DESC test_change +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`test_change`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE test_change RENAME COLUMN a TO a1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "RENAME COLUMN", + "tableName" : "`spark_catalog`.`default`.`test_change`" + } +} + + +-- !query +DESC test_change +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`test_change`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE test_change CHANGE a TYPE STRING +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1245", + "messageParameters" : { + "newName" : "a", + "newType" : "StringType", + "originName" : "a", + "originType" : "IntegerType" + } +} + + +-- !query +DESC test_change +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`test_change`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE test_change CHANGE a AFTER b +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "ALTER COLUMN ... FIRST | ALTER", + "tableName" : "`spark_catalog`.`default`.`test_change`" + } +} + + +-- !query +ALTER TABLE test_change CHANGE b FIRST +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "ALTER COLUMN ... FIRST | ALTER", + "tableName" : "`spark_catalog`.`default`.`test_change`" + } +} + + +-- !query +DESC test_change +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`test_change`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE test_change CHANGE a COMMENT 'this is column a' +-- !query analysis +AlterTableChangeColumnCommand `spark_catalog`.`default`.`test_change`, a, StructField(a,IntegerType,true) + + +-- !query +ALTER TABLE test_change CHANGE b COMMENT '#*02?`' +-- !query analysis +AlterTableChangeColumnCommand `spark_catalog`.`default`.`test_change`, b, StructField(b,StringType,true) + + +-- !query +ALTER TABLE test_change CHANGE c COMMENT '' +-- !query analysis +AlterTableChangeColumnCommand `spark_catalog`.`default`.`test_change`, c, StructField(c,IntegerType,true) + + +-- !query +DESC test_change +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`test_change`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE test_change CHANGE a TYPE INT +-- !query analysis +AlterTableChangeColumnCommand `spark_catalog`.`default`.`test_change`, a, StructField(a,IntegerType,true) + + +-- !query +ALTER TABLE test_change CHANGE a COMMENT 'this is column a' +-- !query analysis +AlterTableChangeColumnCommand `spark_catalog`.`default`.`test_change`, a, StructField(a,IntegerType,true) + + +-- !query +DESC test_change +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`test_change`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE test_change CHANGE invalid_col TYPE INT +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1331", + "messageParameters" : { + "fieldName" : "invalid_col", + "schema" : "root\n |-- a: integer (nullable = true)\n |-- b: string (nullable = true)\n |-- c: integer (nullable = true)\n", + "table" : "spark_catalog.default.test_change" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 51, + "fragment" : "ALTER TABLE test_change CHANGE invalid_col TYPE INT" + } ] +} + + +-- !query +DESC test_change +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`test_change`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE test_change CHANGE A COMMENT 'case insensitivity' +-- !query analysis +AlterTableChangeColumnCommand `spark_catalog`.`default`.`test_change`, a, StructField(a,IntegerType,true) + + +-- !query +DESC test_change +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`test_change`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one" +-- !query analysis +CreateViewCommand `temp_view`, [(a,None), (b,None)], SELECT 1, "one", false, false, LocalTempView, true + +- Project [1 AS 1#x, one AS one#x] + +- OneRowRelation + + +-- !query +ALTER TABLE temp_view CHANGE a TYPE INT +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1013", + "messageParameters" : { + "cmd" : "ALTER TABLE ... CHANGE COLUMN", + "hintStr" : "", + "nameParts" : "temp_view", + "viewStr" : "temp view" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "temp_view" + } ] +} + + +-- !query +CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one" +-- !query analysis +CreateViewCommand `global_temp_view`, [(a,None), (b,None)], SELECT 1, "one", false, false, GlobalTempView, true + +- Project [1 AS 1#x, one AS one#x] + +- OneRowRelation + + +-- !query +ALTER TABLE global_temp.global_temp_view CHANGE a TYPE INT +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1013", + "messageParameters" : { + "cmd" : "ALTER TABLE ... CHANGE COLUMN", + "hintStr" : "", + "nameParts" : "global_temp.global_temp_view", + "viewStr" : "temp view" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 40, + "fragment" : "global_temp.global_temp_view" + } ] +} + + +-- !query +DROP TABLE test_change +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`test_change`, false, false, false + + +-- !query +DROP VIEW global_temp.global_temp_view +-- !query analysis +DropTempViewCommand global_temp.global_temp_view diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out new file mode 100644 index 0000000000000..b41ac81962f92 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out @@ -0,0 +1,694 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create table char_tbl(c char(5), v varchar(6)) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`char_tbl`, false + + +-- !query +desc formatted char_tbl +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_tbl`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +desc formatted char_tbl c +-- !query analysis +DescribeColumnCommand `spark_catalog`.`default`.`char_tbl`, [spark_catalog, default, char_tbl, c], true, [info_name#x, info_value#x] + + +-- !query +show create table char_tbl +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.char_tbl, V1Table(default.char_tbl), [c#x, v#x] + + +-- !query +create table char_tbl2 using parquet as select * from char_tbl +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`char_tbl2`, ErrorIfExists, [c, v] + +- Project [c#x, v#x] + +- SubqueryAlias spark_catalog.default.char_tbl + +- Relation spark_catalog.default.char_tbl[c#x,v#x] parquet + + +-- !query +show create table char_tbl2 +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.char_tbl2, V1Table(default.char_tbl2), [c#x, v#x] + + +-- !query +desc formatted char_tbl2 +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_tbl2`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +desc formatted char_tbl2 c +-- !query analysis +DescribeColumnCommand `spark_catalog`.`default`.`char_tbl2`, [spark_catalog, default, char_tbl2, c], true, [info_name#x, info_value#x] + + +-- !query +create table char_tbl3 like char_tbl +-- !query analysis +CreateTableLikeCommand `char_tbl3`, `char_tbl`, Storage(), false + + +-- !query +desc formatted char_tbl3 +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_tbl3`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +desc formatted char_tbl3 c +-- !query analysis +DescribeColumnCommand `spark_catalog`.`default`.`char_tbl3`, [spark_catalog, default, char_tbl3, c], true, [info_name#x, info_value#x] + + +-- !query +show create table char_tbl3 +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.char_tbl3, V1Table(default.char_tbl3), [c#x, v#x] + + +-- !query +create view char_view as select * from char_tbl +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_ALREADY_EXISTS", + "sqlState" : "42P07", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`char_view`" + } +} + + +-- !query +desc formatted char_view +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_view`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +desc formatted char_view c +-- !query analysis +DescribeColumnCommand `spark_catalog`.`default`.`char_view`, [c], true, [info_name#x, info_value#x] + + +-- !query +show create table char_view +-- !query analysis +ShowCreateTableCommand `spark_catalog`.`default`.`char_view`, [createtab_stmt#x] + + +-- !query +alter table char_tbl rename to char_tbl1 +-- !query analysis +AlterTableRenameCommand `spark_catalog`.`default`.`char_tbl`, `char_tbl1`, false + + +-- !query +desc formatted char_tbl1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_tbl1`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +alter table char_tbl1 change column c type char(6) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1245", + "messageParameters" : { + "newName" : "c", + "newType" : "CharType(6)", + "originName" : "c", + "originType" : "CharType(5)" + } +} + + +-- !query +alter table char_tbl1 change column c type char(5) +-- !query analysis +AlterTableChangeColumnCommand `spark_catalog`.`default`.`char_tbl1`, c, StructField(c,CharType(5),true) + + +-- !query +desc formatted char_tbl1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_tbl1`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +alter table char_tbl1 add columns (d char(5)) +-- !query analysis +AlterTableAddColumnsCommand `spark_catalog`.`default`.`char_tbl1`, [StructField(d,CharType(5),true)] + + +-- !query +desc formatted char_tbl1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_tbl1`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +alter view char_view as select * from char_tbl2 +-- !query analysis +AlterViewAsCommand `spark_catalog`.`default`.`char_view`, select * from char_tbl2, true + +- Project [c#x, v#x] + +- SubqueryAlias spark_catalog.default.char_tbl2 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c#x, 5, true, false, true) AS c#x, v#x] + +- Relation spark_catalog.default.char_tbl2[c#x,v#x] parquet + + +-- !query +desc formatted char_view +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_view`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +alter table char_tbl1 SET TBLPROPERTIES('yes'='no') +-- !query analysis +AlterTableSetPropertiesCommand `spark_catalog`.`default`.`char_tbl1`, [yes=no], false + + +-- !query +desc formatted char_tbl1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_tbl1`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +alter view char_view SET TBLPROPERTIES('yes'='no') +-- !query analysis +AlterTableSetPropertiesCommand `spark_catalog`.`default`.`char_view`, [yes=no], true + + +-- !query +desc formatted char_view +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_view`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +alter table char_tbl1 UNSET TBLPROPERTIES('yes') +-- !query analysis +AlterTableUnsetPropertiesCommand `spark_catalog`.`default`.`char_tbl1`, [yes], false, false + + +-- !query +desc formatted char_tbl1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_tbl1`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +alter view char_view UNSET TBLPROPERTIES('yes') +-- !query analysis +AlterTableUnsetPropertiesCommand `spark_catalog`.`default`.`char_view`, [yes], false, true + + +-- !query +desc formatted char_view +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_view`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +alter table char_tbl1 SET SERDEPROPERTIES('yes'='no') +-- !query analysis +AlterTableSerDePropertiesCommand `spark_catalog`.`default`.`char_tbl1`, Map(yes -> no) + + +-- !query +desc formatted char_tbl1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_tbl1`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +create table char_part(c1 char(5), c2 char(2), v1 varchar(6), v2 varchar(2)) using parquet partitioned by (v2, c2) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +{ + "errorClass" : "TABLE_OR_VIEW_ALREADY_EXISTS", + "sqlState" : "42P07", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`char_part`" + } +} + + +-- !query +desc formatted char_part +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_part`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +alter table char_part add partition (v2='ke', c2='nt') location 'loc1' +-- !query analysis +org.apache.spark.sql.catalyst.analysis.PartitionsAlreadyExistException +{ + "errorClass" : "PARTITIONS_ALREADY_EXIST", + "sqlState" : "428FT", + "messageParameters" : { + "partitionList" : "PARTITION (`v2` = ke, `c2` = nt)", + "tableName" : "`default`.`char_part`" + } +} + + +-- !query +desc formatted char_part +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_part`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +alter table char_part partition (v2='ke') rename to partition (v2='nt') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1232", + "messageParameters" : { + "partitionColumnNames" : "v2, c2", + "specKeys" : "v2", + "tableName" : "`spark_catalog`.`default`.`char_part`" + } +} + + +-- !query +desc formatted char_part +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_part`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +alter table char_part partition (v2='ke', c2='nt') set location 'loc2' +-- !query analysis +AlterTableSetLocationCommand `spark_catalog`.`default`.`char_part`, Map(v2 -> ke, c2 -> nt), loc2 + + +-- !query +desc formatted char_part +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_part`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +MSCK REPAIR TABLE char_part +-- !query analysis +RepairTableCommand `spark_catalog`.`default`.`char_part`, true, false, MSCK REPAIR TABLE + + +-- !query +desc formatted char_part +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`char_part`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +create temporary view str_view as select c, v from values + (null, null), + (null, 'S'), + ('N', 'N '), + ('Ne', 'Sp'), + ('Net ', 'Spa '), + ('NetE', 'Spar'), + ('NetEa ', 'Spark '), + ('NetEas ', 'Spark'), + ('NetEase', 'Spark-') t(c, v) +-- !query analysis +CreateViewCommand `str_view`, select c, v from values + (null, null), + (null, 'S'), + ('N', 'N '), + ('Ne', 'Sp'), + ('Net ', 'Spa '), + ('NetE', 'Spar'), + ('NetEa ', 'Spark '), + ('NetEas ', 'Spark'), + ('NetEase', 'Spark-') t(c, v), false, false, LocalTempView, true + +- Project [c#x, v#x] + +- SubqueryAlias t + +- LocalRelation [c#x, v#x] + + +-- !query +create table char_tbl4(c7 char(7), c8 char(8), v varchar(6), s string) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`char_tbl4`, false + + +-- !query +insert into char_tbl4 select c, c, v, c from str_view +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/char_tbl4, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/char_tbl4], Append, `spark_catalog`.`default`.`char_tbl4`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/char_tbl4), [c7, c8, v, s] ++- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, charTypeWriteSideCheck, cast(c#x as string), 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, charTypeWriteSideCheck, cast(c#x as string), 8, true, false, true) AS c8#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, varcharTypeWriteSideCheck, cast(v#x as string), 6, true, false, true) AS v#x, cast(c#x as string) AS s#x] + +- Project [c#x, c#x, v#x, c#x] + +- SubqueryAlias str_view + +- View (`str_view`, [c#x,v#x]) + +- Project [cast(c#x as string) AS c#x, cast(v#x as string) AS v#x] + +- Project [c#x, v#x] + +- SubqueryAlias t + +- LocalRelation [c#x, v#x] + + +-- !query +select c7, c8, v, s from char_tbl4 +-- !query analysis +Project [c7#x, c8#x, v#x, s#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select c7, c8, v, s from char_tbl4 where c7 = c8 +-- !query analysis +Project [c7#x, c8#x, v#x, s#x] ++- Filter (rpad(c7#x, 8, ) = c8#x) + +- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select c7, c8, v, s from char_tbl4 where c7 = v +-- !query analysis +Project [c7#x, c8#x, v#x, s#x] ++- Filter (c7#x = v#x) + +- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select c7, c8, v, s from char_tbl4 where c7 = s +-- !query analysis +Project [c7#x, c8#x, v#x, s#x] ++- Filter (c7#x = s#x) + +- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select c7, c8, v, s from char_tbl4 where c7 = 'NetEase ' +-- !query analysis +Project [c7#x, c8#x, v#x, s#x] ++- Filter (rpad(c7#x, 22, ) = NetEase ) + +- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select c7, c8, v, s from char_tbl4 where v = 'Spark ' +-- !query analysis +Project [c7#x, c8#x, v#x, s#x] ++- Filter (v#x = Spark ) + +- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select c7, c8, v, s from char_tbl4 order by c7 +-- !query analysis +Sort [c7#x ASC NULLS FIRST], true ++- Project [c7#x, c8#x, v#x, s#x] + +- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select c7, c8, v, s from char_tbl4 order by v +-- !query analysis +Sort [v#x ASC NULLS FIRST], true ++- Project [c7#x, c8#x, v#x, s#x] + +- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select ascii(c7), ascii(c8), ascii(v), ascii(s) from char_tbl4 +-- !query analysis +Project [ascii(c7#x) AS ascii(c7)#x, ascii(c8#x) AS ascii(c8)#x, ascii(v#x) AS ascii(v)#x, ascii(s#x) AS ascii(s)#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select base64(c7), base64(c8), base64(v), ascii(s) from char_tbl4 +-- !query analysis +Project [base64(cast(c7#x as binary)) AS base64(c7)#x, base64(cast(c8#x as binary)) AS base64(c8)#x, base64(cast(v#x as binary)) AS base64(v)#x, ascii(s#x) AS ascii(s)#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select bit_length(c7), bit_length(c8), bit_length(v), bit_length(s) from char_tbl4 +-- !query analysis +Project [bit_length(c7#x) AS bit_length(c7)#x, bit_length(c8#x) AS bit_length(c8)#x, bit_length(v#x) AS bit_length(v)#x, bit_length(s#x) AS bit_length(s)#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select char_length(c7), char_length(c8), char_length(v), char_length(s) from char_tbl4 +-- !query analysis +Project [char_length(c7#x) AS char_length(c7)#x, char_length(c8#x) AS char_length(c8)#x, char_length(v#x) AS char_length(v)#x, char_length(s#x) AS char_length(s)#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select octet_length(c7), octet_length(c8), octet_length(v), octet_length(s) from char_tbl4 +-- !query analysis +Project [octet_length(c7#x) AS octet_length(c7)#x, octet_length(c8#x) AS octet_length(c8)#x, octet_length(v#x) AS octet_length(v)#x, octet_length(s#x) AS octet_length(s)#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select concat_ws('|', c7, c8), concat_ws('|', c7, v), concat_ws('|', c7, s), concat_ws('|', v, s) from char_tbl4 +-- !query analysis +Project [concat_ws(|, c7#x, c8#x) AS concat_ws(|, c7, c8)#x, concat_ws(|, c7#x, v#x) AS concat_ws(|, c7, v)#x, concat_ws(|, c7#x, s#x) AS concat_ws(|, c7, s)#x, concat_ws(|, v#x, s#x) AS concat_ws(|, v, s)#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select concat(c7, c8), concat(c7, v), concat(c7, s), concat(v, s) from char_tbl4 +-- !query analysis +Project [concat(c7#x, c8#x) AS concat(c7, c8)#x, concat(c7#x, v#x) AS concat(c7, v)#x, concat(c7#x, s#x) AS concat(c7, s)#x, concat(v#x, s#x) AS concat(v, s)#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select like(c7, 'Ne _'), like(c8, 'Ne _') from char_tbl4 +-- !query analysis +Project [c7#x LIKE Ne _ AS c7 LIKE Ne _#x, c8#x LIKE Ne _ AS c8 LIKE Ne _#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select like(v, 'Spark_') from char_tbl4 +-- !query analysis +Project [v#x LIKE Spark_ AS v LIKE Spark_#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select c7 = c8, upper(c7) = upper(c8), lower(c7) = lower(c8) from char_tbl4 where s = 'NetEase' +-- !query analysis +Project [(rpad(c7#x, 8, ) = c8#x) AS (c7 = c8)#x, (upper(c7#x) = upper(c8#x)) AS (upper(c7) = upper(c8))#x, (lower(c7#x) = lower(c8#x)) AS (lower(c7) = lower(c8))#x] ++- Filter (s#x = NetEase) + +- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select c7 = s, upper(c7) = upper(s), lower(c7) = lower(s) from char_tbl4 where s = 'NetEase' +-- !query analysis +Project [(c7#x = s#x) AS (c7 = s)#x, (upper(c7#x) = upper(s#x)) AS (upper(c7) = upper(s))#x, (lower(c7#x) = lower(s#x)) AS (lower(c7) = lower(s))#x] ++- Filter (s#x = NetEase) + +- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select c7 = 'NetEase', upper(c7) = upper('NetEase'), lower(c7) = lower('NetEase') from char_tbl4 where s = 'NetEase' +-- !query analysis +Project [(c7#x = NetEase) AS (c7 = NetEase)#x, (upper(c7#x) = upper(NetEase)) AS (upper(c7) = upper(NetEase))#x, (lower(c7#x) = lower(NetEase)) AS (lower(c7) = lower(NetEase))#x] ++- Filter (s#x = NetEase) + +- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select printf('Hey, %s%s%s%s', c7, c8, v, s) from char_tbl4 +-- !query analysis +Project [printf(Hey, %s%s%s%s, c7#x, c8#x, v#x, s#x) AS printf(Hey, %s%s%s%s, c7, c8, v, s)#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select repeat(c7, 2), repeat(c8, 2), repeat(v, 2), repeat(s, 2) from char_tbl4 +-- !query analysis +Project [repeat(c7#x, 2) AS repeat(c7, 2)#x, repeat(c8#x, 2) AS repeat(c8, 2)#x, repeat(v#x, 2) AS repeat(v, 2)#x, repeat(s#x, 2) AS repeat(s, 2)#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select replace(c7, 'Net', 'Apache'), replace(c8, 'Net', 'Apache'), replace(v, 'Spark', 'Kyuubi'), replace(s, 'Net', 'Apache') from char_tbl4 +-- !query analysis +Project [replace(c7#x, Net, Apache) AS replace(c7, Net, Apache)#x, replace(c8#x, Net, Apache) AS replace(c8, Net, Apache)#x, replace(v#x, Spark, Kyuubi) AS replace(v, Spark, Kyuubi)#x, replace(s#x, Net, Apache) AS replace(s, Net, Apache)#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select rpad(c7, 10), rpad(c8, 5), rpad(v, 5), rpad(s, 5) from char_tbl4 +-- !query analysis +Project [rpad(c7#x, 10, ) AS rpad(c7, 10, )#x, rpad(c8#x, 5, ) AS rpad(c8, 5, )#x, rpad(v#x, 5, ) AS rpad(v, 5, )#x, rpad(s#x, 5, ) AS rpad(s, 5, )#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select rtrim(c7), rtrim(c8), rtrim(v), rtrim(s) from char_tbl4 +-- !query analysis +Project [rtrim(c7#x, None) AS rtrim(c7)#x, rtrim(c8#x, None) AS rtrim(c8)#x, rtrim(v#x, None) AS rtrim(v)#x, rtrim(s#x, None) AS rtrim(s)#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select split(c7, 'e'), split(c8, 'e'), split(v, 'a'), split(s, 'e') from char_tbl4 +-- !query analysis +Project [split(c7#x, e, -1) AS split(c7, e, -1)#x, split(c8#x, e, -1) AS split(c8, e, -1)#x, split(v#x, a, -1) AS split(v, a, -1)#x, split(s#x, e, -1) AS split(s, e, -1)#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select substring(c7, 2), substring(c8, 2), substring(v, 3), substring(s, 2) from char_tbl4 +-- !query analysis +Project [substring(c7#x, 2, 2147483647) AS substring(c7, 2, 2147483647)#x, substring(c8#x, 2, 2147483647) AS substring(c8, 2, 2147483647)#x, substring(v#x, 3, 2147483647) AS substring(v, 3, 2147483647)#x, substring(s#x, 2, 2147483647) AS substring(s, 2, 2147483647)#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select left(c7, 2), left(c8, 2), left(v, 3), left(s, 2) from char_tbl4 +-- !query analysis +Project [left(c7#x, 2) AS left(c7, 2)#x, left(c8#x, 2) AS left(c8, 2)#x, left(v#x, 3) AS left(v, 3)#x, left(s#x, 2) AS left(s, 2)#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select right(c7, 2), right(c8, 2), right(v, 3), right(s, 2) from char_tbl4 +-- !query analysis +Project [right(c7#x, 2) AS right(c7, 2)#x, right(c8#x, 2) AS right(c8, 2)#x, right(v#x, 3) AS right(v, 3)#x, right(s#x, 2) AS right(s, 2)#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select typeof(c7), typeof(c8), typeof(v), typeof(s) from char_tbl4 limit 1 +-- !query analysis +GlobalLimit 1 ++- LocalLimit 1 + +- Project [typeof(c7#x) AS typeof(c7)#x, typeof(c8#x) AS typeof(c8)#x, typeof(v#x) AS typeof(v)#x, typeof(s#x) AS typeof(s)#x] + +- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +select cast(c7 as char(1)), cast(c8 as char(10)), cast(v as char(1)), cast(v as varchar(1)), cast(s as char(5)) from char_tbl4 +-- !query analysis +Project [cast(c7#x as string) AS c7#x, cast(c8#x as string) AS c8#x, cast(v#x as string) AS v#x, cast(v#x as string) AS v#x, cast(s#x as string) AS s#x] ++- SubqueryAlias spark_catalog.default.char_tbl4 + +- Project [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c7#x, 7, true, false, true) AS c7#x, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c8#x, 8, true, false, true) AS c8#x, v#x, s#x] + +- Relation spark_catalog.default.char_tbl4[c7#x,c8#x,v#x,s#x] parquet + + +-- !query +drop table char_tbl1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`char_tbl1`, false, false, false + + +-- !query +drop table char_tbl2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`char_tbl2`, false, false, false + + +-- !query +drop table char_tbl3 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`char_tbl3`, false, false, false + + +-- !query +drop table char_tbl4 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`char_tbl4`, false, false, false + + +-- !query +select ascii('§'), ascii('÷'), ascii('×10') +-- !query analysis +Project [ascii(§) AS ascii(§)#x, ascii(÷) AS ascii(÷)#x, ascii(×10) AS ascii(×10)#x] ++- OneRowRelation + + +-- !query +select chr(167), chr(247), chr(215) +-- !query analysis +Project [chr(cast(167 as bigint)) AS chr(167)#x, chr(cast(247 as bigint)) AS chr(247)#x, chr(cast(215 as bigint)) AS chr(215)#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/column-resolution-aggregate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/column-resolution-aggregate.sql.out new file mode 100644 index 0000000000000..eb30443cbae07 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/column-resolution-aggregate.sql.out @@ -0,0 +1,142 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW v1 AS VALUES (1, 1, 1), (2, 2, 1) AS t(a, b, k) +-- !query analysis +CreateViewCommand `v1`, VALUES (1, 1, 1), (2, 2, 1) AS t(a, b, k), false, false, LocalTempView, true + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, k#x] + + +-- !query +CREATE TEMPORARY VIEW v2 AS VALUES (1, 1, 1), (2, 2, 1) AS t(x, y, all) +-- !query analysis +CreateViewCommand `v2`, VALUES (1, 1, 1), (2, 2, 1) AS t(x, y, all), false, false, LocalTempView, true + +- SubqueryAlias t + +- LocalRelation [x#x, y#x, all#x] + + +-- !query +SELECT max(a) AS b, b FROM v1 GROUP BY k +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"b\"", + "expressionAnyValue" : "\"any_value(b)\"" + } +} + + +-- !query +SELECT a FROM v1 WHERE (12, 13) IN (SELECT max(x + 10) AS a, a + 1 FROM v2) +-- !query analysis +Project [a#x] ++- Filter named_struct(_0, 12, _1, 13) IN (list#x []) + : +- Project [a#x, (a#x + 1) AS (lateralAliasReference(a) + 1)#x] + : +- Project [max((x + 10))#x, max((x + 10))#x AS a#x] + : +- Aggregate [max((x#x + 10)) AS max((x + 10))#x] + : +- SubqueryAlias v2 + : +- View (`v2`, [x#x,y#x,all#x]) + : +- Project [cast(x#x as int) AS x#x, cast(y#x as int) AS y#x, cast(all#x as int) AS all#x] + : +- SubqueryAlias t + : +- LocalRelation [x#x, y#x, all#x] + +- SubqueryAlias v1 + +- View (`v1`, [a#x,b#x,k#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(k#x as int) AS k#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, k#x] + + +-- !query +SELECT a AS k FROM v1 GROUP BY k +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT x FROM v2 GROUP BY all +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"x\"", + "expressionAnyValue" : "\"any_value(x)\"" + } +} + + +-- !query +SELECT a AS all, b FROM v1 GROUP BY all +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"b\"", + "expressionAnyValue" : "\"any_value(b)\"" + } +} + + +-- !query +SELECT k AS lca, lca + 1 AS col FROM v1 GROUP BY k, col +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_GROUP_BY", + "sqlState" : "0A000" +} + + +-- !query +SELECT k AS lca, lca + 1 AS col FROM v1 GROUP BY all +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_GROUP_BY", + "sqlState" : "0A000" +} + + +-- !query +SELECT k AS lca, lca + 1 AS col FROM v1 GROUP BY lca +-- !query analysis +Project [lca#x, (lca#x + 1) AS col#x] ++- Project [k#x, k#x AS lca#x] + +- Aggregate [k#x], [k#x] + +- SubqueryAlias v1 + +- View (`v1`, [a#x,b#x,k#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(k#x as int) AS k#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, k#x] + + +-- !query +SELECT * FROM v2 WHERE EXISTS (SELECT a, b FROM v1 GROUP BY all) +-- !query analysis +Project [x#x, y#x, all#x] ++- Filter exists#x [] + : +- Aggregate [a#x, b#x], [a#x, b#x] + : +- SubqueryAlias v1 + : +- View (`v1`, [a#x,b#x,k#x]) + : +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(k#x as int) AS k#x] + : +- SubqueryAlias t + : +- LocalRelation [a#x, b#x, k#x] + +- SubqueryAlias v2 + +- View (`v2`, [x#x,y#x,all#x]) + +- Project [cast(x#x as int) AS x#x, cast(y#x as int) AS y#x, cast(all#x as int) AS all#x] + +- SubqueryAlias t + +- LocalRelation [x#x, y#x, all#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/column-resolution-sort.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/column-resolution-sort.sql.out new file mode 100644 index 0000000000000..a6141f7b579bc --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/column-resolution-sort.sql.out @@ -0,0 +1,60 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW v1 AS VALUES (1, 2, 2), (2, 1, 1) AS t(a, b, k) +-- !query analysis +CreateViewCommand `v1`, VALUES (1, 2, 2), (2, 1, 1) AS t(a, b, k), false, false, LocalTempView, true + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, k#x] + + +-- !query +CREATE TEMPORARY VIEW v2 AS VALUES (1, 2, 2), (2, 1, 1) AS t(a, b, all) +-- !query analysis +CreateViewCommand `v2`, VALUES (1, 2, 2), (2, 1, 1) AS t(a, b, all), false, false, LocalTempView, true + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, all#x] + + +-- !query +SELECT max(a) AS b FROM v1 GROUP BY k ORDER BY b +-- !query analysis +Sort [b#x ASC NULLS FIRST], true ++- Aggregate [k#x], [max(a#x) AS b#x] + +- SubqueryAlias v1 + +- View (`v1`, [a#x,b#x,k#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(k#x as int) AS k#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, k#x] + + +-- !query +SELECT max(a) FROM v2 GROUP BY all ORDER BY all +-- !query analysis +Project [max(a)#x] ++- Sort [all#x ASC NULLS FIRST], true + +- Aggregate [all#x], [max(a#x) AS max(a)#x, all#x] + +- SubqueryAlias v2 + +- View (`v2`, [a#x,b#x,all#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(all#x as int) AS all#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, all#x] + + +-- !query +SELECT (SELECT b FROM v1 ORDER BY all LIMIT 1) FROM v2 +-- !query analysis +Project [scalar-subquery#x [] AS scalarsubquery()#x] +: +- GlobalLimit 1 +: +- LocalLimit 1 +: +- Sort [b#x ASC NULLS FIRST], true +: +- Project [b#x] +: +- SubqueryAlias v1 +: +- View (`v1`, [a#x,b#x,k#x]) +: +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(k#x as int) AS k#x] +: +- SubqueryAlias t +: +- LocalRelation [a#x, b#x, k#x] ++- SubqueryAlias v2 + +- View (`v2`, [a#x,b#x,all#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(all#x as int) AS all#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, all#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution-negative.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution-negative.sql.out new file mode 100644 index 0000000000000..2215a75004baa --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution-negative.sql.out @@ -0,0 +1,482 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE DATABASE mydb1 +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] + + +-- !query +USE mydb1 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] + + +-- !query +CREATE TABLE t1 USING parquet AS SELECT 1 AS i1 +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`mydb1`.`t1`, ErrorIfExists, [i1] + +- Project [1 AS i1#x] + +- OneRowRelation + + +-- !query +CREATE DATABASE mydb2 +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb2] + + +-- !query +USE mydb2 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb2] + + +-- !query +CREATE TABLE t1 USING parquet AS SELECT 20 AS i1 +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`mydb2`.`t1`, ErrorIfExists, [i1] + +- Project [20 AS i1#x] + +- OneRowRelation + + +-- !query +SET spark.sql.crossJoin.enabled = true +-- !query analysis +SetCommand (spark.sql.crossJoin.enabled,Some(true)) + + +-- !query +USE mydb1 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] + + +-- !query +SELECT i1 FROM t1, mydb1.t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`i1`", + "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb1`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 9, + "fragment" : "i1" + } ] +} + + +-- !query +SELECT t1.i1 FROM t1, mydb1.t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`t1`.`i1`", + "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb1`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "t1.i1" + } ] +} + + +-- !query +SELECT mydb1.t1.i1 FROM t1, mydb1.t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`mydb1`.`t1`.`i1`", + "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb1`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "mydb1.t1.i1" + } ] +} + + +-- !query +SELECT i1 FROM t1, mydb2.t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`i1`", + "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 9, + "fragment" : "i1" + } ] +} + + +-- !query +SELECT t1.i1 FROM t1, mydb2.t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`t1`.`i1`", + "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "t1.i1" + } ] +} + + +-- !query +USE mydb2 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb2] + + +-- !query +SELECT i1 FROM t1, mydb1.t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`i1`", + "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 9, + "fragment" : "i1" + } ] +} + + +-- !query +SELECT t1.i1 FROM t1, mydb1.t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`t1`.`i1`", + "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "t1.i1" + } ] +} + + +-- !query +SELECT i1 FROM t1, mydb2.t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`i1`", + "referenceNames" : "[`spark_catalog`.`mydb2`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 9, + "fragment" : "i1" + } ] +} + + +-- !query +SELECT t1.i1 FROM t1, mydb2.t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`t1`.`i1`", + "referenceNames" : "[`spark_catalog`.`mydb2`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "t1.i1" + } ] +} + + +-- !query +SELECT db1.t1.i1 FROM t1, mydb2.t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`db1`.`t1`.`i1`", + "proposal" : "`spark_catalog`.`mydb2`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "db1.t1.i1" + } ] +} + + +-- !query +SET spark.sql.crossJoin.enabled = false +-- !query analysis +SetCommand (spark.sql.crossJoin.enabled,Some(false)) + + +-- !query +USE mydb1 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] + + +-- !query +SELECT mydb1.t1 FROM t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`mydb1`.`t1`", + "proposal" : "`spark_catalog`.`mydb1`.`t1`.`i1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "mydb1.t1" + } ] +} + + +-- !query +SELECT t1.x.y.* FROM t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1051", + "messageParameters" : { + "columns" : "i1", + "targetString" : "t1.x.y" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "t1.x.y.*" + } ] +} + + +-- !query +SELECT t1 FROM mydb1.t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1`", + "proposal" : "`spark_catalog`.`mydb1`.`t1`.`i1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 9, + "fragment" : "t1" + } ] +} + + +-- !query +USE mydb2 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb2] + + +-- !query +SELECT mydb1.t1.i1 FROM t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`mydb1`.`t1`.`i1`", + "proposal" : "`spark_catalog`.`mydb2`.`t1`.`i1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "mydb1.t1.i1" + } ] +} + + +-- !query +USE mydb1 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] + + +-- !query +CREATE VIEW v1 AS SELECT * FROM t1 +-- !query analysis +CreateViewCommand `spark_catalog`.`mydb1`.`v1`, SELECT * FROM t1, false, false, PersistedView, true + +- Project [i1#x] + +- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +DROP TABLE t1 +-- !query analysis +DropTableCommand `spark_catalog`.`mydb1`.`t1`, false, false, false + + +-- !query +CREATE TABLE t1 USING parquet AS SELECT 1 AS i2 +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`mydb1`.`t1`, ErrorIfExists, [i2] + +- Project [1 AS i2#x] + +- OneRowRelation + + +-- !query +SELECT * FROM v1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "i1", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE VIEW spark_catalog.mydb1.v1 AS SELECT * FROM t1", + "viewName" : "`spark_catalog`.`mydb1`.`v1`" + } +} + + +-- !query +USE mydb2 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb2] + + +-- !query +CREATE TEMP VIEW v2 AS SELECT * FROM t1 +-- !query analysis +CreateViewCommand `v2`, SELECT * FROM t1, false, false, LocalTempView, true + +- Project [i1#x] + +- SubqueryAlias spark_catalog.mydb2.t1 + +- Relation spark_catalog.mydb2.t1[i1#x] parquet + + +-- !query +DROP TABLE t1 +-- !query analysis +DropTableCommand `spark_catalog`.`mydb2`.`t1`, false, false, false + + +-- !query +CREATE TABLE t1 USING parquet AS SELECT 1 AS i2 +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`mydb2`.`t1`, ErrorIfExists, [i2] + +- Project [1 AS i2#x] + +- OneRowRelation + + +-- !query +SELECT * FROM v2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "i1", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE TEMPORARY VIEW", + "viewName" : "`v2`" + } +} + + +-- !query +DROP DATABASE mydb1 CASCADE +-- !query analysis +DropNamespace false, true ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] + + +-- !query +DROP DATABASE mydb2 CASCADE +-- !query analysis +DropNamespace false, true ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb2] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution-views.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution-views.sql.out new file mode 100644 index 0000000000000..ea852537903ee --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution-views.sql.out @@ -0,0 +1,174 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW view1 AS SELECT 2 AS i1 +-- !query analysis +CreateViewCommand `view1`, SELECT 2 AS i1, false, true, LocalTempView, true + +- Project [2 AS i1#x] + +- OneRowRelation + + +-- !query +SELECT view1.* FROM view1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias view1 + +- View (`view1`, [i1#x]) + +- Project [cast(i1#x as int) AS i1#x] + +- Project [2 AS i1#x] + +- OneRowRelation + + +-- !query +SELECT * FROM view1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias view1 + +- View (`view1`, [i1#x]) + +- Project [cast(i1#x as int) AS i1#x] + +- Project [2 AS i1#x] + +- OneRowRelation + + +-- !query +SELECT view1.i1 FROM view1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias view1 + +- View (`view1`, [i1#x]) + +- Project [cast(i1#x as int) AS i1#x] + +- Project [2 AS i1#x] + +- OneRowRelation + + +-- !query +SELECT i1 FROM view1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias view1 + +- View (`view1`, [i1#x]) + +- Project [cast(i1#x as int) AS i1#x] + +- Project [2 AS i1#x] + +- OneRowRelation + + +-- !query +SELECT a.i1 FROM view1 AS a +-- !query analysis +Project [i1#x] ++- SubqueryAlias a + +- SubqueryAlias view1 + +- View (`view1`, [i1#x]) + +- Project [cast(i1#x as int) AS i1#x] + +- Project [2 AS i1#x] + +- OneRowRelation + + +-- !query +SELECT i1 FROM view1 AS a +-- !query analysis +Project [i1#x] ++- SubqueryAlias a + +- SubqueryAlias view1 + +- View (`view1`, [i1#x]) + +- Project [cast(i1#x as int) AS i1#x] + +- Project [2 AS i1#x] + +- OneRowRelation + + +-- !query +DROP VIEW view1 +-- !query analysis +DropTempViewCommand view1 + + +-- !query +CREATE OR REPLACE GLOBAL TEMPORARY VIEW view1 as SELECT 1 as i1 +-- !query analysis +CreateViewCommand `view1`, SELECT 1 as i1, false, true, GlobalTempView, true + +- Project [1 AS i1#x] + +- OneRowRelation + + +-- !query +SELECT * FROM global_temp.view1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias global_temp.view1 + +- View (`global_temp`.`view1`, [i1#x]) + +- Project [cast(i1#x as int) AS i1#x] + +- Project [1 AS i1#x] + +- OneRowRelation + + +-- !query +SELECT global_temp.view1.* FROM global_temp.view1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias global_temp.view1 + +- View (`global_temp`.`view1`, [i1#x]) + +- Project [cast(i1#x as int) AS i1#x] + +- Project [1 AS i1#x] + +- OneRowRelation + + +-- !query +SELECT i1 FROM global_temp.view1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias global_temp.view1 + +- View (`global_temp`.`view1`, [i1#x]) + +- Project [cast(i1#x as int) AS i1#x] + +- Project [1 AS i1#x] + +- OneRowRelation + + +-- !query +SELECT global_temp.view1.i1 FROM global_temp.view1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias global_temp.view1 + +- View (`global_temp`.`view1`, [i1#x]) + +- Project [cast(i1#x as int) AS i1#x] + +- Project [1 AS i1#x] + +- OneRowRelation + + +-- !query +SELECT view1.i1 FROM global_temp.view1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias global_temp.view1 + +- View (`global_temp`.`view1`, [i1#x]) + +- Project [cast(i1#x as int) AS i1#x] + +- Project [1 AS i1#x] + +- OneRowRelation + + +-- !query +SELECT a.i1 FROM global_temp.view1 AS a +-- !query analysis +Project [i1#x] ++- SubqueryAlias a + +- SubqueryAlias global_temp.view1 + +- View (`global_temp`.`view1`, [i1#x]) + +- Project [cast(i1#x as int) AS i1#x] + +- Project [1 AS i1#x] + +- OneRowRelation + + +-- !query +SELECT i1 FROM global_temp.view1 AS a +-- !query analysis +Project [i1#x] ++- SubqueryAlias a + +- SubqueryAlias global_temp.view1 + +- View (`global_temp`.`view1`, [i1#x]) + +- Project [cast(i1#x as int) AS i1#x] + +- Project [1 AS i1#x] + +- OneRowRelation + + +-- !query +DROP VIEW global_temp.view1 +-- !query analysis +DropTempViewCommand global_temp.view1 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution.sql.out new file mode 100644 index 0000000000000..6a2542c2aeb6a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution.sql.out @@ -0,0 +1,442 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE DATABASE mydb1 +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] + + +-- !query +USE mydb1 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] + + +-- !query +CREATE TABLE t1 USING parquet AS SELECT 1 AS i1 +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`mydb1`.`t1`, ErrorIfExists, [i1] + +- Project [1 AS i1#x] + +- OneRowRelation + + +-- !query +CREATE DATABASE mydb2 +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb2] + + +-- !query +USE mydb2 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb2] + + +-- !query +CREATE TABLE t1 USING parquet AS SELECT 20 AS i1 +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`mydb2`.`t1`, ErrorIfExists, [i1] + +- Project [20 AS i1#x] + +- OneRowRelation + + +-- !query +USE mydb1 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] + + +-- !query +SELECT i1 FROM t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +SELECT i1 FROM mydb1.t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +SELECT t1.i1 FROM t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +SELECT t1.i1 FROM mydb1.t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +SELECT mydb1.t1.i1 FROM t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +SELECT mydb1.t1.i1 FROM mydb1.t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +USE mydb2 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb2] + + +-- !query +SELECT i1 FROM t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb2.t1 + +- Relation spark_catalog.mydb2.t1[i1#x] parquet + + +-- !query +SELECT i1 FROM mydb1.t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +SELECT t1.i1 FROM t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb2.t1 + +- Relation spark_catalog.mydb2.t1[i1#x] parquet + + +-- !query +SELECT t1.i1 FROM mydb1.t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +SELECT mydb1.t1.i1 FROM mydb1.t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +USE mydb1 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] + + +-- !query +SELECT t1.* FROM t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +SELECT mydb1.t1.* FROM mydb1.t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +SELECT t1.* FROM mydb1.t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +USE mydb2 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb2] + + +-- !query +SELECT t1.* FROM t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb2.t1 + +- Relation spark_catalog.mydb2.t1[i1#x] parquet + + +-- !query +SELECT mydb1.t1.* FROM mydb1.t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +SELECT t1.* FROM mydb1.t1 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +SELECT a.* FROM mydb1.t1 AS a +-- !query analysis +Project [i1#x] ++- SubqueryAlias a + +- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +USE mydb1 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] + + +-- !query +CREATE TABLE t3 USING parquet AS SELECT * FROM VALUES (4,1), (3,1) AS t3(c1, c2) +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`mydb1`.`t3`, ErrorIfExists, [c1, c2] + +- Project [c1#x, c2#x] + +- SubqueryAlias t3 + +- LocalRelation [c1#x, c2#x] + + +-- !query +CREATE TABLE t4 USING parquet AS SELECT * FROM VALUES (4,1), (2,1) AS t4(c2, c3) +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`mydb1`.`t4`, ErrorIfExists, [c2, c3] + +- Project [c2#x, c3#x] + +- SubqueryAlias t4 + +- LocalRelation [c2#x, c3#x] + + +-- !query +SELECT * FROM t3 WHERE c1 IN (SELECT c2 FROM t4 WHERE t4.c3 = t3.c2) +-- !query analysis +Project [c1#x, c2#x] ++- Filter c1#x IN (list#x [c2#x]) + : +- Project [c2#x] + : +- Filter (c3#x = outer(c2#x)) + : +- SubqueryAlias spark_catalog.mydb1.t4 + : +- Relation spark_catalog.mydb1.t4[c2#x,c3#x] parquet + +- SubqueryAlias spark_catalog.mydb1.t3 + +- Relation spark_catalog.mydb1.t3[c1#x,c2#x] parquet + + +-- !query +SELECT * FROM mydb1.t3 WHERE c1 IN + (SELECT mydb1.t4.c2 FROM mydb1.t4 WHERE mydb1.t4.c3 = mydb1.t3.c2) +-- !query analysis +Project [c1#x, c2#x] ++- Filter c1#x IN (list#x [c2#x]) + : +- Project [c2#x] + : +- Filter (c3#x = outer(c2#x)) + : +- SubqueryAlias spark_catalog.mydb1.t4 + : +- Relation spark_catalog.mydb1.t4[c2#x,c3#x] parquet + +- SubqueryAlias spark_catalog.mydb1.t3 + +- Relation spark_catalog.mydb1.t3[c1#x,c2#x] parquet + + +-- !query +SET spark.sql.crossJoin.enabled = true +-- !query analysis +SetCommand (spark.sql.crossJoin.enabled,Some(true)) + + +-- !query +SELECT mydb1.t1.i1 FROM t1, mydb2.t1 +-- !query analysis +Project [i1#x] ++- Join Inner + :- SubqueryAlias spark_catalog.mydb1.t1 + : +- Relation spark_catalog.mydb1.t1[i1#x] parquet + +- SubqueryAlias spark_catalog.mydb2.t1 + +- Relation spark_catalog.mydb2.t1[i1#x] parquet + + +-- !query +SELECT mydb1.t1.i1 FROM mydb1.t1, mydb2.t1 +-- !query analysis +Project [i1#x] ++- Join Inner + :- SubqueryAlias spark_catalog.mydb1.t1 + : +- Relation spark_catalog.mydb1.t1[i1#x] parquet + +- SubqueryAlias spark_catalog.mydb2.t1 + +- Relation spark_catalog.mydb2.t1[i1#x] parquet + + +-- !query +USE mydb2 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb2] + + +-- !query +SELECT mydb1.t1.i1 FROM t1, mydb1.t1 +-- !query analysis +Project [i1#x] ++- Join Inner + :- SubqueryAlias spark_catalog.mydb2.t1 + : +- Relation spark_catalog.mydb2.t1[i1#x] parquet + +- SubqueryAlias spark_catalog.mydb1.t1 + +- Relation spark_catalog.mydb1.t1[i1#x] parquet + + +-- !query +SET spark.sql.crossJoin.enabled = false +-- !query analysis +SetCommand (spark.sql.crossJoin.enabled,Some(false)) + + +-- !query +USE mydb1 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] + + +-- !query +CREATE TABLE t5(i1 INT, t5 STRUCT) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`mydb1`.`t5`, false + + +-- !query +INSERT INTO t5 VALUES(1, (2, 3)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/mydb1.db/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/mydb1.db/t5], Append, `spark_catalog`.`mydb1`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/mydb1.db/t5), [i1, t5] ++- Project [cast(col1#x as int) AS i1#x, cast(col2#x as struct) AS t5#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t5.i1 FROM t5 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t5 + +- Relation spark_catalog.mydb1.t5[i1#x,t5#x] parquet + + +-- !query +SELECT t5.t5.i1 FROM t5 +-- !query analysis +Project [t5#x.i1 AS i1#x] ++- SubqueryAlias spark_catalog.mydb1.t5 + +- Relation spark_catalog.mydb1.t5[i1#x,t5#x] parquet + + +-- !query +SELECT t5.t5.i1 FROM mydb1.t5 +-- !query analysis +Project [t5#x.i1 AS i1#x] ++- SubqueryAlias spark_catalog.mydb1.t5 + +- Relation spark_catalog.mydb1.t5[i1#x,t5#x] parquet + + +-- !query +SELECT t5.i1 FROM mydb1.t5 +-- !query analysis +Project [i1#x] ++- SubqueryAlias spark_catalog.mydb1.t5 + +- Relation spark_catalog.mydb1.t5[i1#x,t5#x] parquet + + +-- !query +SELECT t5.* FROM mydb1.t5 +-- !query analysis +Project [i1#x, t5#x] ++- SubqueryAlias spark_catalog.mydb1.t5 + +- Relation spark_catalog.mydb1.t5[i1#x,t5#x] parquet + + +-- !query +SELECT t5.t5.* FROM mydb1.t5 +-- !query analysis +Project [t5#x.i1 AS i1#x, t5#x.i2 AS i2#x] ++- SubqueryAlias spark_catalog.mydb1.t5 + +- Relation spark_catalog.mydb1.t5[i1#x,t5#x] parquet + + +-- !query +SELECT mydb1.t5.t5.i1 FROM mydb1.t5 +-- !query analysis +Project [t5#x.i1 AS i1#x] ++- SubqueryAlias spark_catalog.mydb1.t5 + +- Relation spark_catalog.mydb1.t5[i1#x,t5#x] parquet + + +-- !query +SELECT mydb1.t5.t5.i2 FROM mydb1.t5 +-- !query analysis +Project [t5#x.i2 AS i2#x] ++- SubqueryAlias spark_catalog.mydb1.t5 + +- Relation spark_catalog.mydb1.t5[i1#x,t5#x] parquet + + +-- !query +SELECT mydb1.t5.* FROM mydb1.t5 +-- !query analysis +Project [i1#x, t5#x] ++- SubqueryAlias spark_catalog.mydb1.t5 + +- Relation spark_catalog.mydb1.t5[i1#x,t5#x] parquet + + +-- !query +SELECT mydb1.t5.* FROM t5 +-- !query analysis +Project [i1#x, t5#x] ++- SubqueryAlias spark_catalog.mydb1.t5 + +- Relation spark_catalog.mydb1.t5[i1#x,t5#x] parquet + + +-- !query +USE default +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [default] + + +-- !query +DROP DATABASE mydb1 CASCADE +-- !query analysis +DropNamespace false, true ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] + + +-- !query +DROP DATABASE mydb2 CASCADE +-- !query analysis +DropNamespace false, true ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb2] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/comments.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/comments.sql.out new file mode 100644 index 0000000000000..30df13ace66bc --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/comments.sql.out @@ -0,0 +1,145 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +/* This is the first example of bracketed comment. +SELECT 'ommented out content' AS first; +*/ +SELECT 'selected content' AS first +-- !query analysis +Project [selected content AS first#x] ++- OneRowRelation + + +-- !query +/* This is the second example of bracketed comment. +SELECT '/', 'ommented out content' AS second; +*/ +SELECT '/', 'selected content' AS second +-- !query analysis +Project [/ AS /#x, selected content AS second#x] ++- OneRowRelation + + +-- !query +/* This is the third example of bracketed comment. + *SELECT '*', 'ommented out content' AS third; + */ +SELECT '*', 'selected content' AS third +-- !query analysis +Project [* AS *#x, selected content AS third#x] ++- OneRowRelation + + +-- !query +/**/ +SELECT 'selected content' AS fourth +-- !query analysis +Project [selected content AS fourth#x] ++- OneRowRelation + + +-- !query +/* This is the first example of nested bracketed comment. +/* I am a nested bracketed comment.*/ +*/ +SELECT 'selected content' AS fifth +-- !query analysis +Project [selected content AS fifth#x] ++- OneRowRelation + + +-- !query +/* This is the second example of nested bracketed comment. +/* I am a nested bracketed comment. + */ + */ +SELECT 'selected content' AS sixth +-- !query analysis +Project [selected content AS sixth#x] ++- OneRowRelation + + +-- !query +/* + * This is the third example of nested bracketed comment. + /* + * I am a nested bracketed comment. + */ + */ +SELECT 'selected content' AS seventh +-- !query analysis +Project [selected content AS seventh#x] ++- OneRowRelation + + +-- !query +/* + * This is the fourth example of nested bracketed comment. +SELECT /* I am a nested bracketed comment.*/ * FROM testData; + */ +SELECT 'selected content' AS eighth +-- !query analysis +Project [selected content AS eighth#x] ++- OneRowRelation + + +-- !query +SELECT /* + * This is the fifth example of nested bracketed comment. +/* I am a second level of nested bracketed comment. +/* I am a third level of nested bracketed comment. +Other information of third level. +SELECT 'ommented out content' AS ninth; +*/ +Other information of second level. +*/ +Other information of first level. +*/ +'selected content' AS ninth +-- !query analysis +Project [selected content AS ninth#x] ++- OneRowRelation + + +-- !query +/*/**/*/ +SELECT 'selected content' AS tenth +-- !query analysis +Project [selected content AS tenth#x] ++- OneRowRelation + + +-- !query +/*abc*/ +select 1 as a +/* + +2 as b +/*abc*/ +, 3 as c + +/**/ +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNCLOSED_BRACKETED_COMMENT", + "sqlState" : "42601" +} + + +-- !query +/*abc*/ +select 1 as a +/* + +2 as b +/*abc*/ +, 3 as c + +/**/ +select 4 as d +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNCLOSED_BRACKETED_COMMENT", + "sqlState" : "42601" +} 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 new file mode 100644 index 0000000000000..022c260ac6f60 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/comparator.sql.out @@ -0,0 +1,69 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select x'00' < x'0f' +-- !query analysis +Project [(0x00 < 0x0F) AS (X'00' < X'0F')#x] ++- OneRowRelation + + +-- !query +select x'00' < x'ff' +-- !query analysis +Project [(0x00 < 0xFF) AS (X'00' < X'FF')#x] ++- OneRowRelation + + +-- !query +select '1 ' = 1Y +-- !query analysis +Project [(cast(1 as tinyint) = 1) AS (1 = 1)#x] ++- OneRowRelation + + +-- !query +select '\t1 ' = 1Y +-- !query analysis +Project [(cast( 1 as tinyint) = 1) AS ( 1 = 1)#x] ++- OneRowRelation + + +-- !query +select '1 ' = 1S +-- !query analysis +Project [(cast(1 as smallint) = 1) AS (1 = 1)#x] ++- OneRowRelation + + +-- !query +select '1 ' = 1 +-- !query analysis +Project [(cast(1 as int) = 1) AS (1 = 1)#x] ++- OneRowRelation + + +-- !query +select ' 1' = 1L +-- !query analysis +Project [(cast( 1 as bigint) = 1) AS ( 1 = 1)#x] ++- OneRowRelation + + +-- !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] ++- OneRowRelation + + +-- !query +select ' 1.0 ' = 1.0D +-- !query analysis +Project [(cast( 1.0 as double) = 1.0) AS ( 1.0 = 1.0)#x] ++- OneRowRelation + + +-- !query +select ' 1.0 ' = 1.0BD +-- !query analysis +Project [(cast( 1.0 as double) = cast(1.0 as double)) AS ( 1.0 = 1.0)#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/count.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/count.sql.out new file mode 100644 index 0000000000000..29af8f2a549e8 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/count.sql.out @@ -0,0 +1,230 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b) +-- !query analysis +CreateViewCommand `testData`, SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b), false, true, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT + count(*), count(1), count(null), count(a), count(b), count(a + b), count((a, b)) +FROM testData +-- !query analysis +Aggregate [count(1) AS count(1)#xL, count(1) AS count(1)#xL, count(null) AS count(NULL)#xL, count(a#x) AS count(a)#xL, count(b#x) AS count(b)#xL, count((a#x + b#x)) AS count((a + b))#xL, count(named_struct(a, a#x, b, b#x)) AS count(named_struct(a, a, b, b))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT + count(DISTINCT 1), + count(DISTINCT null), + count(DISTINCT a), + count(DISTINCT b), + count(DISTINCT (a + b)), + count(DISTINCT (a, b)) +FROM testData +-- !query analysis +Aggregate [count(distinct 1) AS count(DISTINCT 1)#xL, count(distinct null) AS count(DISTINCT NULL)#xL, count(distinct a#x) AS count(DISTINCT a)#xL, count(distinct b#x) AS count(DISTINCT b)#xL, count(distinct (a#x + b#x)) AS count(DISTINCT (a + b))#xL, count(distinct named_struct(a, a#x, b, b#x)) AS count(DISTINCT named_struct(a, a, b, b))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT count(a, b), count(b, a), count(testData.*, testData.*) FROM testData +-- !query analysis +Aggregate [count(a#x, b#x) AS count(a, b)#xL, count(b#x, a#x) AS count(b, a)#xL, count(a#x, b#x, a#x, b#x) AS count(a, b, a, b)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT + count(DISTINCT a, b), count(DISTINCT b, a), count(DISTINCT *), count(DISTINCT testData.*, testData.*) +FROM testData +-- !query analysis +Aggregate [count(distinct a#x, b#x) AS count(DISTINCT a, b)#xL, count(distinct b#x, a#x) AS count(DISTINCT b, a)#xL, count(distinct a#x, b#x) AS count(DISTINCT a, b)#xL, count(distinct a#x, b#x, a#x, b#x) AS count(DISTINCT a, b, a, b)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT count(DISTINCT 3,2) +-- !query analysis +Aggregate [count(distinct 3, 2) AS count(DISTINCT 3, 2)#xL] ++- OneRowRelation + + +-- !query +SELECT count(DISTINCT 2), count(DISTINCT 2,3) +-- !query analysis +Aggregate [count(distinct 2) AS count(DISTINCT 2)#xL, count(distinct 2, 3) AS count(DISTINCT 2, 3)#xL] ++- OneRowRelation + + +-- !query +SELECT count(DISTINCT 2), count(DISTINCT 3,2) +-- !query analysis +Aggregate [count(distinct 2) AS count(DISTINCT 2)#xL, count(distinct 3, 2) AS count(DISTINCT 3, 2)#xL] ++- OneRowRelation + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 2,3) FROM testData +-- !query analysis +Aggregate [count(distinct a#x) AS count(DISTINCT a)#xL, count(distinct 2, 3) AS count(DISTINCT 2, 3)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 3,2) FROM testData +-- !query analysis +Aggregate [count(distinct a#x) AS count(DISTINCT a)#xL, count(distinct 3, 2) AS count(DISTINCT 3, 2)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 2), count(DISTINCT 2,3) FROM testData +-- !query analysis +Aggregate [count(distinct a#x) AS count(DISTINCT a)#xL, count(distinct 2) AS count(DISTINCT 2)#xL, count(distinct 2, 3) AS count(DISTINCT 2, 3)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 2), count(DISTINCT 3,2) FROM testData +-- !query analysis +Aggregate [count(distinct a#x) AS count(DISTINCT a)#xL, count(distinct 2) AS count(DISTINCT 2)#xL, count(distinct 3, 2) AS count(DISTINCT 3, 2)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT count(distinct 0.8), percentile_approx(distinct a, 0.8) FROM testData +-- !query analysis +Aggregate [count(distinct 0.8) AS count(DISTINCT 0.8)#xL, percentile_approx(distinct a#x, cast(0.8 as double), 10000, 0, 0) AS percentile_approx(DISTINCT a, 0.8, 10000)#x] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +set spark.sql.legacy.allowParameterlessCount=true +-- !query analysis +SetCommand (spark.sql.legacy.allowParameterlessCount,Some(true)) + + +-- !query +SELECT count() FROM testData +-- !query analysis +Aggregate [count() AS count()#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +set spark.sql.legacy.allowParameterlessCount=false +-- !query analysis +SetCommand (spark.sql.legacy.allowParameterlessCount,Some(false)) + + +-- !query +SELECT count() FROM testData +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITH_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "expectedNum" : " >= 1", + "functionName" : "`count`", + "legacyConfKey" : "\"spark.sql.legacy.allowParameterlessCount\"", + "legacyConfValue" : "\"true\"", + "legacyNum" : "0" + } +} + + +-- !query +set spark.sql.legacy.allowStarWithSingleTableIdentifierInCount=true +-- !query analysis +SetCommand (spark.sql.legacy.allowStarWithSingleTableIdentifierInCount,Some(true)) + + +-- !query +SELECT count(testData.*) FROM testData +-- !query analysis +Aggregate [count(a#x, b#x) AS count(a, b)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +set spark.sql.legacy.allowStarWithSingleTableIdentifierInCount=false +-- !query analysis +SetCommand (spark.sql.legacy.allowStarWithSingleTableIdentifierInCount,Some(false)) + + +-- !query +SELECT count(testData.*) FROM testData +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1021", + "messageParameters" : { + "targetString" : "testData" + } +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cross-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cross-join.sql.out new file mode 100644 index 0000000000000..7d318cd949b89 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cross-join.sql.out @@ -0,0 +1,259 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query analysis +CreateViewCommand `nt1`, select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1), false, false, LocalTempView, true + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- LocalRelation [k#x, v1#x] + + +-- !query +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query analysis +CreateViewCommand `nt2`, select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2), false, false, LocalTempView, true + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 cross join nt2 +-- !query analysis +Project [k#x, v1#x, k#x, v2#x] ++- Join Cross + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 cross join nt2 where nt1.k = nt2.k +-- !query analysis +Project [k#x, v1#x, k#x, v2#x] ++- Filter (k#x = k#x) + +- Join Cross + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 cross join nt2 on (nt1.k = nt2.k) +-- !query analysis +Project [k#x, v1#x, k#x, v2#x] ++- Join Cross, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 cross join nt2 where nt1.v1 = 1 and nt2.v2 = 22 +-- !query analysis +Project [k#x, v1#x, k#x, v2#x] ++- Filter ((v1#x = 1) AND (v2#x = 22)) + +- Join Cross + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT a.key, b.key FROM +(SELECT k key FROM nt1 WHERE v1 < 2) a +CROSS JOIN +(SELECT k key FROM nt2 WHERE v2 = 22) b +-- !query analysis +Project [key#x, key#x] ++- Join Cross + :- SubqueryAlias a + : +- Project [k#x AS key#x] + : +- Filter (v1#x < 2) + : +- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias b + +- Project [k#x AS key#x] + +- Filter (v2#x = 22) + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +create temporary view A(a, va) as select * from nt1 +-- !query analysis +CreateViewCommand `A`, [(a,None), (va,None)], select * from nt1, false, false, LocalTempView, true + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- View (`nt1`, [k#x,v1#x]) + +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- LocalRelation [k#x, v1#x] + + +-- !query +create temporary view B(b, vb) as select * from nt1 +-- !query analysis +CreateViewCommand `B`, [(b,None), (vb,None)], select * from nt1, false, false, LocalTempView, true + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- View (`nt1`, [k#x,v1#x]) + +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- LocalRelation [k#x, v1#x] + + +-- !query +create temporary view C(c, vc) as select * from nt1 +-- !query analysis +CreateViewCommand `C`, [(c,None), (vc,None)], select * from nt1, false, false, LocalTempView, true + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- View (`nt1`, [k#x,v1#x]) + +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- LocalRelation [k#x, v1#x] + + +-- !query +create temporary view D(d, vd) as select * from nt1 +-- !query analysis +CreateViewCommand `D`, [(d,None), (vd,None)], select * from nt1, false, false, LocalTempView, true + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- View (`nt1`, [k#x,v1#x]) + +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- LocalRelation [k#x, v1#x] + + +-- !query +select * from ((A join B on (a = b)) cross join C) join D on (a = d) +-- !query analysis +Project [a#x, va#x, b#x, vb#x, c#x, vc#x, d#x, vd#x] ++- Join Inner, (a#x = d#x) + :- Join Cross + : :- Join Inner, (a#x = b#x) + : : :- SubqueryAlias a + : : : +- View (`A`, [a#x,va#x]) + : : : +- Project [cast(k#x as string) AS a#x, cast(v1#x as int) AS va#x] + : : : +- Project [k#x, v1#x] + : : : +- SubqueryAlias nt1 + : : : +- View (`nt1`, [k#x,v1#x]) + : : : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : : : +- Project [k#x, v1#x] + : : : +- SubqueryAlias nt1 + : : : +- LocalRelation [k#x, v1#x] + : : +- SubqueryAlias b + : : +- View (`B`, [b#x,vb#x]) + : : +- Project [cast(k#x as string) AS b#x, cast(v1#x as int) AS vb#x] + : : +- Project [k#x, v1#x] + : : +- SubqueryAlias nt1 + : : +- View (`nt1`, [k#x,v1#x]) + : : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : : +- Project [k#x, v1#x] + : : +- SubqueryAlias nt1 + : : +- LocalRelation [k#x, v1#x] + : +- SubqueryAlias c + : +- View (`C`, [c#x,vc#x]) + : +- Project [cast(k#x as string) AS c#x, cast(v1#x as int) AS vc#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias d + +- View (`D`, [d#x,vd#x]) + +- Project [cast(k#x as string) AS d#x, cast(v1#x as int) AS vd#x] + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- View (`nt1`, [k#x,v1#x]) + +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- LocalRelation [k#x, v1#x] + + +-- !query +SELECT * FROM nt1 CROSS JOIN nt2 ON (nt1.k > nt2.k) +-- !query analysis +Project [k#x, v1#x, k#x, v2#x] ++- Join Cross, (k#x > k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] 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 new file mode 100644 index 0000000000000..e59f247d944dd --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/csv-functions.sql.out @@ -0,0 +1,258 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select from_csv('1, 3.14', 'a INT, f FLOAT') +-- !query analysis +Project [from_csv(StructField(a,IntegerType,true), StructField(f,FloatType,true), 1, 3.14, Some(America/Los_Angeles), None) AS from_csv(1, 3.14)#x] ++- OneRowRelation + + +-- !query +select from_csv('26/08/2015', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')) +-- !query analysis +Project [from_csv(StructField(time,TimestampType,true), (timestampFormat,dd/MM/yyyy), 26/08/2015, Some(America/Los_Angeles), None) AS from_csv(26/08/2015)#x] ++- OneRowRelation + + +-- !query +select from_csv('1', 1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_SCHEMA.NON_STRING_LITERAL", + "sqlState" : "42K07", + "messageParameters" : { + "inputSchema" : "\"1\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "from_csv('1', 1)" + } ] +} + + +-- !query +select from_csv('1', 'a InvalidType') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'InvalidType'", + "hint" : ": extra input 'InvalidType'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "from_csv('1', 'a InvalidType')" + } ] +} + + +-- !query +select from_csv('1', 'Array') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_SCHEMA.NON_STRUCT_TYPE", + "sqlState" : "42K07", + "messageParameters" : { + "dataType" : "\"ARRAY\"", + "inputSchema" : "\"Array\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "from_csv('1', 'Array')" + } ] +} + + +-- !query +select from_csv('1', 'a INT', named_struct('mode', 'PERMISSIVE')) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_MAP_FUNCTION", + "sqlState" : "42K06", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 65, + "fragment" : "from_csv('1', 'a INT', named_struct('mode', 'PERMISSIVE'))" + } ] +} + + +-- !query +select from_csv('1', 'a INT', map('mode', 1)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_STRING_TYPE", + "sqlState" : "42K06", + "messageParameters" : { + "mapType" : "\"MAP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "from_csv('1', 'a INT', map('mode', 1))" + } ] +} + + +-- !query +select from_csv() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3]", + "functionName" : "`from_csv`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "from_csv()" + } ] +} + + +-- !query +select from_csv('1,abc', schema_of_csv('1,abc')) +-- !query analysis +Project [from_csv(StructField(_c0,IntegerType,true), StructField(_c1,StringType,true), 1,abc, Some(America/Los_Angeles), None) AS from_csv(1,abc)#x] ++- OneRowRelation + + +-- !query +select schema_of_csv('1|abc', map('delimiter', '|')) +-- !query analysis +Project [schema_of_csv(1|abc, (delimiter,|)) AS schema_of_csv(1|abc)#x] ++- OneRowRelation + + +-- !query +select schema_of_csv(null) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "csv", + "sqlExpr" : "\"schema_of_csv(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "schema_of_csv(null)" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW csvTable(csvField, a) AS SELECT * FROM VALUES ('1,abc', 'a') +-- !query analysis +CreateViewCommand `csvTable`, [(csvField,None), (a,None)], SELECT * FROM VALUES ('1,abc', 'a'), false, false, LocalTempView, true + +- Project [col1#x, col2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT schema_of_csv(csvField) FROM csvTable +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"csvField\"", + "inputName" : "csv", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"schema_of_csv(csvField)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "schema_of_csv(csvField)" + } ] +} + + +-- !query +DROP VIEW IF EXISTS csvTable +-- !query analysis +DropTempViewCommand csvTable + + +-- !query +select to_csv(named_struct('a', 1, 'b', 2)) +-- !query analysis +Project [to_csv(named_struct(a, 1, b, 2), Some(America/Los_Angeles)) AS to_csv(named_struct(a, 1, b, 2))#x] ++- OneRowRelation + + +-- !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] ++- OneRowRelation + + +-- !query +select to_csv(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_MAP_FUNCTION", + "sqlState" : "42K06", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "to_csv(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE'))" + } ] +} + + +-- !query +select to_csv(named_struct('a', 1, 'b', 2), map('mode', 1)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_STRING_TYPE", + "sqlState" : "42K06", + "messageParameters" : { + "mapType" : "\"MAP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "to_csv(named_struct('a', 1, 'b', 2), map('mode', 1))" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-legacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-legacy.sql.out new file mode 100644 index 0000000000000..6006c0a47ee98 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-legacy.sql.out @@ -0,0 +1,462 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t +-- !query analysis +Project [1#x] ++- SubqueryAlias t + +- Project [1#x] + +- SubqueryAlias t2 + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +) +-- !query analysis +Aggregate [max(c#x) AS max(c)#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [c#x] + +- SubqueryAlias t + +- Project [1#x AS c#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +) +-- !query analysis +Project [scalar-subquery#x [] AS scalarsubquery()#x] +: +- Project [1#x] +: +- SubqueryAlias t +: +- Project [1 AS 1#x] +: +- OneRowRelation ++- OneRowRelation + + +-- !query +SELECT * FROM + ( + WITH cte AS (SELECT * FROM range(10)) + SELECT * FROM cte WHERE id = 8 + ) a +UNION +SELECT * FROM cte +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 120, + "stopIndex" : 122, + "fragment" : "cte" + } ] +} + + +-- !query +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2 +-- !query analysis +Project [1#x] ++- SubqueryAlias t2 + +- Project [1#x] + +- SubqueryAlias t + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2 +-- !query analysis +Project [scalarsubquery()#x] ++- SubqueryAlias t2 + +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : +- Aggregate [max(c#x) AS max(c)#x] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [c#x] + : +- SubqueryAlias t + : +- Project [1#x AS c#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- OneRowRelation + + +-- !query +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2 +-- !query analysis +Project [2#x] ++- SubqueryAlias t2 + +- Project [2#x] + +- SubqueryAlias t2 + +- Project [2#x] + +- SubqueryAlias t + +- Project [2 AS 2#x] + +- OneRowRelation + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query analysis +Aggregate [max(c#x) AS max(c)#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [c#x] + +- SubqueryAlias t + +- Project [2#x AS c#x] + +- Project [2 AS 2#x] + +- OneRowRelation + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query analysis +Aggregate [sum(c#x) AS sum(c)#xL] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [max(c#x) AS c#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [c#x] + +- SubqueryAlias t + +- Project [2#x AS c#x] + +- Project [2 AS 2#x] + +- OneRowRelation + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query analysis +Aggregate [sum(c#x) AS sum(c)#xL] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [max(c#x) AS c#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [c#x] + +- SubqueryAlias t + +- Project [3#x AS c#x] + +- Project [3 AS 3#x] + +- OneRowRelation + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +) +-- !query analysis +Project [scalar-subquery#x [] AS scalarsubquery()#x] +: +- Project [1#x] +: +- SubqueryAlias t +: +- Project [1 AS 1#x] +: +- OneRowRelation ++- OneRowRelation + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query analysis +Project [scalar-subquery#x [] AS scalarsubquery()#x] +: +- Project [scalar-subquery#x [] AS scalarsubquery()#x] +: : +- Project [1#x] +: : +- SubqueryAlias t +: : +- Project [1 AS 1#x] +: : +- OneRowRelation +: +- OneRowRelation ++- OneRowRelation + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query analysis +Project [scalar-subquery#x [] AS scalarsubquery()#x] +: +- Project [scalar-subquery#x [] AS scalarsubquery()#x] +: : +- Project [1#x] +: : +- SubqueryAlias t +: : +- Project [1 AS 1#x] +: : +- OneRowRelation +: +- OneRowRelation ++- OneRowRelation + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT * FROM t +WHERE c IN ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query analysis +Project [c#x] ++- Filter c#x IN (list#x []) + : +- Project [c#x] + : +- SubqueryAlias t + : +- Project [1#x AS c#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- SubqueryAlias t + +- Project [1#x AS c#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +WITH + t AS ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 + ), + t2 AS (SELECT 2) +SELECT * FROM t +-- !query analysis +Project [1#x] ++- SubqueryAlias t + +- Project [1#x] + +- SubqueryAlias t2 + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +WITH + abc AS (SELECT 1), + t AS ( + WITH aBc AS (SELECT 2) + SELECT * FROM aBC + ) +SELECT * FROM t +-- !query analysis +Project [1#x] ++- SubqueryAlias t + +- Project [1#x] + +- SubqueryAlias abc + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +WITH abc AS (SELECT 1) +SELECT ( + WITH aBc AS (SELECT 2) + SELECT * FROM aBC +) +-- !query analysis +Project [scalar-subquery#x [] AS scalarsubquery()#x] +: +- Project [1#x] +: +- SubqueryAlias abc +: +- Project [1 AS 1#x] +: +- OneRowRelation ++- OneRowRelation + + +-- !query +WITH + t1 AS (SELECT 1), + t2 AS ( + WITH t3 AS ( + SELECT * FROM t1 + ) + SELECT * FROM t3 + ) +SELECT * FROM t2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 73, + "stopIndex" : 74, + "fragment" : "t1" + } ] +} + + +-- !query +WITH cte_outer AS ( + SELECT 1 +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_outer + ) + SELECT * FROM cte_inner +) +-- !query analysis +Project [1#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [1#x] + +- SubqueryAlias cte_inner + +- Project [1#x] + +- SubqueryAlias cte_outer + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +WITH cte_outer AS ( + SELECT 1 +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM ( + WITH cte_inner_inner AS ( + SELECT * FROM cte_outer + ) + SELECT * FROM cte_inner_inner + ) + ) + SELECT * FROM cte_inner +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte_outer`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 146, + "stopIndex" : 154, + "fragment" : "cte_outer" + } ] +} + + +-- !query +WITH cte_outer AS ( + WITH cte_invisible_inner AS ( + SELECT 1 + ) + SELECT * FROM cte_invisible_inner +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_invisible_inner + ) + SELECT * FROM cte_inner +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 164, + "stopIndex" : 182, + "fragment" : "cte_invisible_inner" + } ] +} + + +-- !query +WITH cte_outer AS ( + SELECT * FROM ( + WITH cte_invisible_inner AS ( + SELECT 1 + ) + SELECT * FROM cte_invisible_inner + ) +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_invisible_inner + ) + SELECT * FROM cte_inner +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 194, + "stopIndex" : 212, + "fragment" : "cte_invisible_inner" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out new file mode 100644 index 0000000000000..cda60d34a4a7b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out @@ -0,0 +1,555 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t2 +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x] +: +- SubqueryAlias t2 +: +- CTERelationRef xxxx, true, [1#x] ++- Project [1#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [1#x] + + +-- !query +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +) +-- !query analysis +Aggregate [max(c#x) AS max(c)#x] ++- SubqueryAlias __auto_generated_subquery_name + +- WithCTE + :- CTERelationDef xxxx, false + : +- SubqueryAlias t + : +- Project [1#x AS c#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [c#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [c#x] + + +-- !query +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +) +-- !query analysis +Project [scalar-subquery#x [] AS scalarsubquery()#x] +: +- WithCTE +: :- CTERelationDef xxxx, false +: : +- SubqueryAlias t +: : +- Project [1 AS 1#x] +: : +- OneRowRelation +: +- Project [1#x] +: +- SubqueryAlias t +: +- CTERelationRef xxxx, true, [1#x] ++- OneRowRelation + + +-- !query +SELECT * FROM + ( + WITH cte AS (SELECT * FROM range(10)) + SELECT * FROM cte WHERE id = 8 + ) a +UNION +SELECT * FROM cte +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 120, + "stopIndex" : 122, + "fragment" : "cte" + } ] +} + + +-- !query +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "t" + } +} + + +-- !query +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x AS c#x] +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t2 +: +- Project [scalar-subquery#x [] AS scalarsubquery()#x] +: : +- Aggregate [max(c#x) AS max(c)#x] +: : +- SubqueryAlias __auto_generated_subquery_name +: : +- WithCTE +: : :- CTERelationDef xxxx, false +: : : +- SubqueryAlias t +: : : +- Project [2#x AS c#x] +: : : +- Project [2 AS 2#x] +: : : +- OneRowRelation +: : +- Project [c#x] +: : +- SubqueryAlias t +: : +- CTERelationRef xxxx, true, [c#x] +: +- OneRowRelation ++- Project [scalarsubquery()#x] + +- SubqueryAlias t2 + +- CTERelationRef xxxx, true, [scalarsubquery()#x] + + +-- !query +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "t" + } +} + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x AS c#x] +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [2#x AS c#x] +: +- Project [2 AS 2#x] +: +- OneRowRelation ++- Aggregate [max(c#x) AS max(c)#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [c#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [c#x] + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x AS c#x] +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [2#x AS c#x] +: +- Project [2 AS 2#x] +: +- OneRowRelation ++- Aggregate [sum(c#x) AS sum(c)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [max(c#x) AS c#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [c#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [c#x] + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x AS c#x] +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [2#x AS c#x] +: +- Project [2 AS 2#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [3#x AS c#x] +: +- Project [3 AS 3#x] +: +- OneRowRelation ++- Aggregate [sum(c#x) AS sum(c)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [max(c#x) AS c#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [c#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [c#x] + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "t" + } +} + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "t" + } +} + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "t" + } +} + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT * FROM t +WHERE c IN ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "t" + } +} + + +-- !query +WITH + t AS ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 + ), + t2 AS (SELECT 2) +SELECT * FROM t +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t2 +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x] +: +- SubqueryAlias t2 +: +- CTERelationRef xxxx, true, [1#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias t2 +: +- Project [2 AS 2#x] +: +- OneRowRelation ++- Project [1#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [1#x] + + +-- !query +WITH + abc AS (SELECT 1), + t AS ( + WITH aBc AS (SELECT 2) + SELECT * FROM aBC + ) +SELECT * FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "aBc" + } +} + + +-- !query +WITH abc AS (SELECT 1) +SELECT ( + WITH aBc AS (SELECT 2) + SELECT * FROM aBC +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1200", + "messageParameters" : { + "config" : "spark.sql.legacy.ctePrecedencePolicy", + "name" : "aBc" + } +} + + +-- !query +WITH + t1 AS (SELECT 1), + t2 AS ( + WITH t3 AS ( + SELECT * FROM t1 + ) + SELECT * FROM t3 + ) +SELECT * FROM t2 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t1 +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t3 +: +- Project [1#x] +: +- SubqueryAlias t1 +: +- CTERelationRef xxxx, true, [1#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias t2 +: +- Project [1#x] +: +- SubqueryAlias t3 +: +- CTERelationRef xxxx, true, [1#x] ++- Project [1#x] + +- SubqueryAlias t2 + +- CTERelationRef xxxx, true, [1#x] + + +-- !query +WITH cte_outer AS ( + SELECT 1 +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_outer + ) + SELECT * FROM cte_inner +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte_outer +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte_inner +: +- Project [1#x] +: +- SubqueryAlias cte_outer +: +- CTERelationRef xxxx, true, [1#x] ++- Project [1#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [1#x] + +- SubqueryAlias cte_inner + +- CTERelationRef xxxx, true, [1#x] + + +-- !query +WITH cte_outer AS ( + SELECT 1 +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM ( + WITH cte_inner_inner AS ( + SELECT * FROM cte_outer + ) + SELECT * FROM cte_inner_inner + ) + ) + SELECT * FROM cte_inner +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte_outer +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte_inner_inner +: +- Project [1#x] +: +- SubqueryAlias cte_outer +: +- CTERelationRef xxxx, true, [1#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte_inner +: +- Project [1#x] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [1#x] +: +- SubqueryAlias cte_inner_inner +: +- CTERelationRef xxxx, true, [1#x] ++- Project [1#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [1#x] + +- SubqueryAlias cte_inner + +- CTERelationRef xxxx, true, [1#x] + + +-- !query +WITH cte_outer AS ( + WITH cte_invisible_inner AS ( + SELECT 1 + ) + SELECT * FROM cte_invisible_inner +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_invisible_inner + ) + SELECT * FROM cte_inner +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 164, + "stopIndex" : 182, + "fragment" : "cte_invisible_inner" + } ] +} + + +-- !query +WITH cte_outer AS ( + SELECT * FROM ( + WITH cte_invisible_inner AS ( + SELECT 1 + ) + SELECT * FROM cte_invisible_inner + ) +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_invisible_inner + ) + SELECT * FROM cte_inner +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 194, + "stopIndex" : 212, + "fragment" : "cte_invisible_inner" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out new file mode 100644 index 0000000000000..689bffd58e2dc --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out @@ -0,0 +1,639 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t2 +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x] +: +- SubqueryAlias t2 +: +- CTERelationRef xxxx, true, [1#x] ++- Project [1#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [1#x] + + +-- !query +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +) +-- !query analysis +Aggregate [max(c#x) AS max(c)#x] ++- SubqueryAlias __auto_generated_subquery_name + +- WithCTE + :- CTERelationDef xxxx, false + : +- SubqueryAlias t + : +- Project [1#x AS c#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [c#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [c#x] + + +-- !query +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +) +-- !query analysis +Project [scalar-subquery#x [] AS scalarsubquery()#x] +: +- WithCTE +: :- CTERelationDef xxxx, false +: : +- SubqueryAlias t +: : +- Project [1 AS 1#x] +: : +- OneRowRelation +: +- Project [1#x] +: +- SubqueryAlias t +: +- CTERelationRef xxxx, true, [1#x] ++- OneRowRelation + + +-- !query +SELECT * FROM + ( + WITH cte AS (SELECT * FROM range(10)) + SELECT * FROM cte WHERE id = 8 + ) a +UNION +SELECT * FROM cte +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 120, + "stopIndex" : 122, + "fragment" : "cte" + } ] +} + + +-- !query +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [2 AS 2#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t2 +: +- Project [2#x] +: +- SubqueryAlias t +: +- CTERelationRef xxxx, true, [2#x] ++- Project [2#x] + +- SubqueryAlias t2 + +- CTERelationRef xxxx, true, [2#x] + + +-- !query +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x AS c#x] +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t2 +: +- Project [scalar-subquery#x [] AS scalarsubquery()#x] +: : +- Aggregate [max(c#x) AS max(c)#x] +: : +- SubqueryAlias __auto_generated_subquery_name +: : +- WithCTE +: : :- CTERelationDef xxxx, false +: : : +- SubqueryAlias t +: : : +- Project [2#x AS c#x] +: : : +- Project [2 AS 2#x] +: : : +- OneRowRelation +: : +- Project [c#x] +: : +- SubqueryAlias t +: : +- CTERelationRef xxxx, true, [c#x] +: +- OneRowRelation ++- Project [scalarsubquery()#x] + +- SubqueryAlias t2 + +- CTERelationRef xxxx, true, [scalarsubquery()#x] + + +-- !query +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [2 AS 2#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [3 AS 3#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t2 +: +- Project [3#x] +: +- SubqueryAlias t +: +- CTERelationRef xxxx, true, [3#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias t2 +: +- Project [3#x] +: +- SubqueryAlias t2 +: +- CTERelationRef xxxx, true, [3#x] ++- Project [3#x] + +- SubqueryAlias t2 + +- CTERelationRef xxxx, true, [3#x] + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x AS c#x] +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [2#x AS c#x] +: +- Project [2 AS 2#x] +: +- OneRowRelation ++- Aggregate [max(c#x) AS max(c)#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [c#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [c#x] + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x AS c#x] +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [2#x AS c#x] +: +- Project [2 AS 2#x] +: +- OneRowRelation ++- Aggregate [sum(c#x) AS sum(c)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [max(c#x) AS c#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [c#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [c#x] + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x AS c#x] +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [2#x AS c#x] +: +- Project [2 AS 2#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [3#x AS c#x] +: +- Project [3 AS 3#x] +: +- OneRowRelation ++- Aggregate [sum(c#x) AS sum(c)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [max(c#x) AS c#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [c#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [c#x] + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1 AS 1#x] +: +- OneRowRelation ++- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : +- WithCTE + : :- CTERelationDef xxxx, false + : : +- SubqueryAlias t + : : +- Project [2 AS 2#x] + : : +- OneRowRelation + : +- Project [2#x] + : +- SubqueryAlias t + : +- CTERelationRef xxxx, true, [2#x] + +- OneRowRelation + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1 AS 1#x] +: +- OneRowRelation ++- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- WithCTE + : : :- CTERelationDef xxxx, false + : : : +- SubqueryAlias t + : : : +- Project [2 AS 2#x] + : : : +- OneRowRelation + : : +- Project [2#x] + : : +- SubqueryAlias t + : : +- CTERelationRef xxxx, true, [2#x] + : +- OneRowRelation + +- OneRowRelation + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1 AS 1#x] +: +- OneRowRelation ++- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : +- WithCTE + : :- CTERelationDef xxxx, false + : : +- SubqueryAlias t + : : +- Project [2 AS 2#x] + : : +- OneRowRelation + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- WithCTE + : : :- CTERelationDef xxxx, false + : : : +- SubqueryAlias t + : : : +- Project [3 AS 3#x] + : : : +- OneRowRelation + : : +- Project [3#x] + : : +- SubqueryAlias t + : : +- CTERelationRef xxxx, true, [3#x] + : +- OneRowRelation + +- OneRowRelation + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT * FROM t +WHERE c IN ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x AS c#x] +: +- Project [1 AS 1#x] +: +- OneRowRelation ++- Project [c#x] + +- Filter c#x IN (list#x []) + : +- WithCTE + : :- CTERelationDef xxxx, false + : : +- SubqueryAlias t + : : +- Project [2#x AS c#x] + : : +- Project [2 AS 2#x] + : : +- OneRowRelation + : +- Project [c#x] + : +- SubqueryAlias t + : +- CTERelationRef xxxx, true, [c#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [c#x] + + +-- !query +WITH + t AS ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 + ), + t2 AS (SELECT 2) +SELECT * FROM t +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t2 +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x] +: +- SubqueryAlias t2 +: +- CTERelationRef xxxx, true, [1#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias t2 +: +- Project [2 AS 2#x] +: +- OneRowRelation ++- Project [1#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [1#x] + + +-- !query +WITH + abc AS (SELECT 1), + t AS ( + WITH aBc AS (SELECT 2) + SELECT * FROM aBC + ) +SELECT * FROM t +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias abc +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias aBc +: +- Project [2 AS 2#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [2#x] +: +- SubqueryAlias aBC +: +- CTERelationRef xxxx, true, [2#x] ++- Project [2#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [2#x] + + +-- !query +WITH abc AS (SELECT 1) +SELECT ( + WITH aBc AS (SELECT 2) + SELECT * FROM aBC +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias abc +: +- Project [1 AS 1#x] +: +- OneRowRelation ++- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : +- WithCTE + : :- CTERelationDef xxxx, false + : : +- SubqueryAlias aBc + : : +- Project [2 AS 2#x] + : : +- OneRowRelation + : +- Project [2#x] + : +- SubqueryAlias aBC + : +- CTERelationRef xxxx, true, [2#x] + +- OneRowRelation + + +-- !query +WITH + t1 AS (SELECT 1), + t2 AS ( + WITH t3 AS ( + SELECT * FROM t1 + ) + SELECT * FROM t3 + ) +SELECT * FROM t2 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t1 +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias t3 +: +- Project [1#x] +: +- SubqueryAlias t1 +: +- CTERelationRef xxxx, true, [1#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias t2 +: +- Project [1#x] +: +- SubqueryAlias t3 +: +- CTERelationRef xxxx, true, [1#x] ++- Project [1#x] + +- SubqueryAlias t2 + +- CTERelationRef xxxx, true, [1#x] + + +-- !query +WITH cte_outer AS ( + SELECT 1 +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_outer + ) + SELECT * FROM cte_inner +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte_outer +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte_inner +: +- Project [1#x] +: +- SubqueryAlias cte_outer +: +- CTERelationRef xxxx, true, [1#x] ++- Project [1#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [1#x] + +- SubqueryAlias cte_inner + +- CTERelationRef xxxx, true, [1#x] + + +-- !query +WITH cte_outer AS ( + SELECT 1 +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM ( + WITH cte_inner_inner AS ( + SELECT * FROM cte_outer + ) + SELECT * FROM cte_inner_inner + ) + ) + SELECT * FROM cte_inner +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte_outer +: +- Project [1 AS 1#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte_inner_inner +: +- Project [1#x] +: +- SubqueryAlias cte_outer +: +- CTERelationRef xxxx, true, [1#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte_inner +: +- Project [1#x] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [1#x] +: +- SubqueryAlias cte_inner_inner +: +- CTERelationRef xxxx, true, [1#x] ++- Project [1#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [1#x] + +- SubqueryAlias cte_inner + +- CTERelationRef xxxx, true, [1#x] + + +-- !query +WITH cte_outer AS ( + WITH cte_invisible_inner AS ( + SELECT 1 + ) + SELECT * FROM cte_invisible_inner +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_invisible_inner + ) + SELECT * FROM cte_inner +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 164, + "stopIndex" : 182, + "fragment" : "cte_invisible_inner" + } ] +} + + +-- !query +WITH cte_outer AS ( + SELECT * FROM ( + WITH cte_invisible_inner AS ( + SELECT 1 + ) + SELECT * FROM cte_invisible_inner + ) +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_invisible_inner + ) + SELECT * FROM cte_inner +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 194, + "stopIndex" : 212, + "fragment" : "cte_invisible_inner" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out new file mode 100644 index 0000000000000..0347e48cd5f81 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out @@ -0,0 +1,280 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t as select * from values 0, 1, 2 as t(id) +-- !query analysis +CreateViewCommand `t`, select * from values 0, 1, 2 as t(id), false, false, LocalTempView, true + +- Project [id#x] + +- SubqueryAlias t + +- LocalRelation [id#x] + + +-- !query +create temporary view t2 as select * from values 0, 1 as t(id) +-- !query analysis +CreateViewCommand `t2`, select * from values 0, 1 as t(id), false, false, LocalTempView, true + +- Project [id#x] + +- SubqueryAlias t + +- LocalRelation [id#x] + + +-- !query +WITH s AS (SELECT 1 FROM s) SELECT * FROM s +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`s`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 26, + "fragment" : "s" + } ] +} + + +-- !query +WITH r AS (SELECT (SELECT * FROM r)) +SELECT * FROM r +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`r`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 34, + "fragment" : "r" + } ] +} + + +-- !query +WITH t AS (SELECT 1 FROM t) SELECT * FROM t +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1 AS 1#x] +: +- SubqueryAlias t +: +- View (`t`, [id#x]) +: +- Project [cast(id#x as int) AS id#x] +: +- Project [id#x] +: +- SubqueryAlias t +: +- LocalRelation [id#x] ++- Project [1#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [1#x] + + +-- !query +WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`s2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 28, + "fragment" : "s2" + } ] +} + + +-- !query +WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1 cross join t2 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t1 +: +- Project [id#x] +: +- SubqueryAlias t2 +: +- View (`t2`, [id#x]) +: +- Project [cast(id#x as int) AS id#x] +: +- Project [id#x] +: +- SubqueryAlias t +: +- LocalRelation [id#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias t2 +: +- Project [2 AS 2#x] +: +- SubqueryAlias t1 +: +- CTERelationRef xxxx, true, [id#x] ++- Project [id#x, 2#x] + +- Join Cross + :- SubqueryAlias t1 + : +- CTERelationRef xxxx, true, [id#x] + +- SubqueryAlias t2 + +- CTERelationRef xxxx, true, [2#x] + + +-- !query +WITH CTE1 AS ( + SELECT b.id AS id + FROM T2 a + CROSS JOIN (SELECT id AS id FROM T2) b +) +SELECT t1.id AS c1, + t2.id AS c2 +FROM CTE1 t1 + CROSS JOIN CTE1 t2 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias CTE1 +: +- Project [id#x AS id#x] +: +- Join Cross +: :- SubqueryAlias a +: : +- SubqueryAlias t2 +: : +- View (`t2`, [id#x]) +: : +- Project [cast(id#x as int) AS id#x] +: : +- Project [id#x] +: : +- SubqueryAlias t +: : +- LocalRelation [id#x] +: +- SubqueryAlias b +: +- Project [id#x AS id#x] +: +- SubqueryAlias t2 +: +- View (`t2`, [id#x]) +: +- Project [cast(id#x as int) AS id#x] +: +- Project [id#x] +: +- SubqueryAlias t +: +- LocalRelation [id#x] ++- Project [id#x AS c1#x, id#x AS c2#x] + +- Join Cross + :- SubqueryAlias t1 + : +- SubqueryAlias CTE1 + : +- CTERelationRef xxxx, true, [id#x] + +- SubqueryAlias t2 + +- SubqueryAlias CTE1 + +- CTERelationRef xxxx, true, [id#x] + + +-- !query +WITH t(x) AS (SELECT 1) +SELECT * FROM t WHERE x = 1 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x AS x#x] +: +- Project [1 AS 1#x] +: +- OneRowRelation ++- Project [x#x] + +- Filter (x#x = 1) + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [x#x] + + +-- !query +WITH t(x, y) AS (SELECT 1, 2) +SELECT * FROM t WHERE x = 1 AND y = 2 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x AS x#x, 2#x AS y#x] +: +- Project [1 AS 1#x, 2 AS 2#x] +: +- OneRowRelation ++- Project [x#x, y#x] + +- Filter ((x#x = 1) AND (y#x = 2)) + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [x#x, y#x] + + +-- !query +WITH t(x, x) AS (SELECT 1, 2) +SELECT * FROM t +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1#x AS x#x, 2#x AS x#x] +: +- Project [1 AS 1#x, 2 AS 2#x] +: +- OneRowRelation ++- Project [x#x, x#x] + +- SubqueryAlias t + +- CTERelationRef xxxx, true, [x#x, x#x] + + +-- !query +WITH t() AS (SELECT 1) +SELECT * FROM t +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "')'", + "hint" : "" + } +} + + +-- !query +WITH + t(x) AS (SELECT 1), + t(x) AS (SELECT 2) +SELECT * FROM t +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0038", + "messageParameters" : { + "duplicateNames" : "'t'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "WITH\n t(x) AS (SELECT 1),\n t(x) AS (SELECT 2)\nSELECT * FROM t" + } ] +} + + +-- !query +WITH t AS (SELECT 1 FROM non_existing_table) +SELECT 2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`non_existing_table`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 43, + "fragment" : "non_existing_table" + } ] +} + + +-- !query +DROP VIEW IF EXISTS t +-- !query analysis +DropTempViewCommand t + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query analysis +DropTempViewCommand t2 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/current_database_catalog.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/current_database_catalog.sql.out new file mode 100644 index 0000000000000..ad72e19b6bb7f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/current_database_catalog.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select current_database(), current_schema(), current_catalog() +-- !query analysis +Project [current_database() AS current_database()#x, current_database() AS current_database()#x, current_catalog() AS current_catalog()#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 new file mode 100644 index 0000000000000..baa7cc3554087 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/date.sql.out @@ -0,0 +1,942 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view date_view as select '2011-11-11' date_str, '1' int_str +-- !query analysis +CreateViewCommand `date_view`, select '2011-11-11' date_str, '1' int_str, false, false, LocalTempView, true + +- Project [2011-11-11 AS date_str#x, 1 AS int_str#x] + +- OneRowRelation + + +-- !query +select date '2019-01-01\t' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2020-01-01中文' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2020-01-01中文'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "date '2020-01-01中文'" + } ] +} + + +-- !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] ++- OneRowRelation + + +-- !query +select make_date(2000, 13, 1) +-- !query analysis +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, false) AS make_date(2000, 1, 33)#x] ++- OneRowRelation + + +-- !query +select date'015' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'015'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "date'015'" + } ] +} + + +-- !query +select date'2021-4294967297-11' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-4294967297-11'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "date'2021-4294967297-11'" + } ] +} + + +-- !query +select current_date = current_date +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select current_date() = current_date() +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select curdate(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "0", + "functionName" : "`curdate`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "curdate(1)" + } ] +} + + +-- !query +select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null) +-- !query analysis +Project [date_from_unix_date(0) AS date_from_unix_date(0)#x, date_from_unix_date(1000) AS date_from_unix_date(1000)#x, date_from_unix_date(cast(null as int)) AS date_from_unix_date(NULL)#x] ++- OneRowRelation + + +-- !query +select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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)) AS to_date(NULL)#x, to_date(2016-12-31, None, Some(America/Los_Angeles)) AS to_date(2016-12-31)#x, to_date(2016-12-31, Some(yyyy-MM-dd), Some(America/Los_Angeles)) 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)) 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)) AS to_date(02-29, MM-dd)#x] ++- OneRowRelation + + +-- !query +select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), + dayofweek('1582-10-15 13:10:15'), dayofweek(timestamp_ltz'1582-10-15 13:10:15'), dayofweek(timestamp_ntz'1582-10-15 13:10:15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), + weekday('1582-10-15 13:10:15'), weekday(timestamp_ltz'1582-10-15 13:10:15'), weekday(timestamp_ntz'1582-10-15 13:10:15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select year('1500-01-01'), year('1582-10-15 13:10:15'), year(timestamp_ltz'1582-10-15 13:10:15'), year(timestamp_ntz'1582-10-15 13:10:15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select month('1500-01-01'), month('1582-10-15 13:10:15'), month(timestamp_ltz'1582-10-15 13:10:15'), month(timestamp_ntz'1582-10-15 13:10:15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(timestamp_ltz'1582-10-15 13:10:15'), dayOfYear(timestamp_ntz'1582-10-15 13:10:15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon") +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- OneRowRelation + + +-- !query +select next_day("xx", "Mon") +-- !query analysis +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, false) 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] ++- OneRowRelation + + +-- !query +select date_add(date'2011-11-11', 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- OneRowRelation + + +-- !query +select date_add('2011-11-11', 1Y) +-- !query analysis +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', 1S) +-- !query analysis +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', 1L) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "date_add('2011-11-11', 1L)" + } ] +} + + +-- !query +select date_add('2011-11-11', 1.0) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_add('2011-11-11', 1.0)" + } ] +} + + +-- !query +select date_add('2011-11-11', 1E1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_add('2011-11-11', 1E1)" + } ] +} + + +-- !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] ++- 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" + } +} + + +-- !query +select date_add(null, 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_add(date'2011-11-11', null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_add(timestamp_ltz'2011-11-11 12:12:12', 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_add(timestamp_ntz'2011-11-11 12:12:12', 1) +-- !query analysis +Project [date_add(cast(2011-11-11 12:12:12 as date), 1) AS date_add(TIMESTAMP_NTZ '2011-11-11 12:12:12', 1)#x] ++- OneRowRelation + + +-- !query +select date_sub(date'2011-11-11', 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_sub('2011-11-11', 1) +-- !query analysis +Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x] ++- OneRowRelation + + +-- !query +select date_sub('2011-11-11', 1Y) +-- !query analysis +Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x] ++- OneRowRelation + + +-- !query +select date_sub('2011-11-11', 1S) +-- !query analysis +Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x] ++- OneRowRelation + + +-- !query +select date_sub('2011-11-11', 1L) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "date_sub('2011-11-11', 1L)" + } ] +} + + +-- !query +select date_sub('2011-11-11', 1.0) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_sub('2011-11-11', 1.0)" + } ] +} + + +-- !query +select date_sub('2011-11-11', 1E1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_sub('2011-11-11', 1E1)" + } ] +} + + +-- !query +select date_sub(date'2011-11-11', '1') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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" + } +} + + +-- !query +select date_sub(null, 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_sub(date'2011-11-11', null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_sub(timestamp_ltz'2011-11-11 12:12:12', 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_sub(timestamp_ntz'2011-11-11 12:12:12', 1) +-- !query analysis +Project [date_sub(cast(2011-11-11 12:12:12 as date), 1) AS date_sub(TIMESTAMP_NTZ '2011-11-11 12:12:12', 1)#x] ++- OneRowRelation + + +-- !query +select date_add('2011-11-11', int_str) from date_view +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"int_str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "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 +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"int_str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "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 +select date_add(date_str, 1) from date_view +-- !query analysis +Project [date_add(cast(date_str#x as date), 1) AS date_add(date_str, 1)#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(date_str, 1) from date_view +-- !query analysis +Project [date_sub(cast(date_str#x as date), 1) AS date_sub(date_str, 1)#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 '2011-11-11' + 1E1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "date '2011-11-11' + 1E1" + } ] +} + + +-- !query +select date '2001-09-28' + 7Y +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select 7S + date '2001-09-28' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2001-10-01' - 7 +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2001-10-01' - date '2001-09-28' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2001-10-01' - '2001-09-28' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2001-09-28\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "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 +select '2001-10-01' - date '2001-09-28' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2001-09-28' - null +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select null - date '2019-10-06' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_str - date '2001-09-28' from date_view +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2001-09-28' - date_str from date_view +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"date_str\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "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 +select date'2011-11-11' + '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "date'2011-11-11' + '1'" + } ] +} + + +-- !query +select '1' + date'2011-11-11' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' + date'2011-11-11'" + } ] +} + + +-- !query +select date'2011-11-11' + null +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select null + date'2011-11-11' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2012-01-01' - interval '2-2' year to month, + date '2011-11-11' - interval '2' day, + date '2012-01-01' + interval '-2-2' year to month, + date '2011-11-11' + interval '-2' month, + - interval '2-2' year to month + date '2012-01-01', + interval '-2' day + date '2011-11-11' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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)) AS to_date(26/October/2015, dd/MMMMM/yyyy)#x] ++- OneRowRelation + + +-- !query +select from_json('{"d":"26/October/2015"}', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_json(StructField(d,DateType,true), (dateFormat,dd/MMMMM/yyyy), {"d":"26/October/2015"}, Some(America/Los_Angeles)) AS from_json({"d":"26/October/2015"})#x] ++- OneRowRelation + + +-- !query +select from_csv('26/October/2015', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_csv(StructField(d,DateType,true), (dateFormat,dd/MMMMM/yyyy), 26/October/2015, Some(America/Los_Angeles), None) AS from_csv(26/October/2015)#x] ++- OneRowRelation + + +-- !query +select dateadd(MICROSECOND, 1001, timestamp'2022-02-25 01:02:03.123') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(MILLISECOND, -1, timestamp'2022-02-25 01:02:03.456') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(SECOND, 58, timestamp'2022-02-25 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(MINUTE, -100, date'2022-02-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(HOUR, -1, timestamp'2022-02-25 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(DAY, 367, date'2022-02-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(WEEK, -4, timestamp'2022-02-25 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(MONTH, -1, timestamp'2022-02-25 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(QUARTER, 5, date'2022-02-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(YEAR, 1, date'2022-02-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(MICROSECOND, timestamp'2022-02-25 01:02:03.123', timestamp'2022-02-25 01:02:03.124001') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(MILLISECOND, timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(SECOND, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 01:03:01') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(MINUTE, date'2022-02-25', timestamp'2022-02-24 22:20:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(HOUR, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 00:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(DAY, date'2022-02-25', timestamp'2023-02-27 00:00:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(WEEK, timestamp'2022-02-25 01:02:03', timestamp'2022-01-28 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(MONTH, timestamp'2022-02-25 01:02:03', timestamp'2022-01-25 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(QUARTER, date'2022-02-25', date'2023-05-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(YEAR, date'2022-02-25', date'2023-02-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting-invalid.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting-invalid.sql.out new file mode 100644 index 0000000000000..6a7888bbd501c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting-invalid.sql.out @@ -0,0 +1,258 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select date_format('2018-11-17 13:33:33.333', 'GGGGG') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), GGGGG, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, GGGGG)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'yyyyyyy') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), yyyyyyy, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, yyyyyyy)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'qqqqq') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), qqqqq, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, qqqqq)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'QQQQQ') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), QQQQQ, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, QQQQQ)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'MMMMM') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), MMMMM, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, MMMMM)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'LLLLL') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), LLLLL, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, LLLLL)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'EEEEE') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), EEEEE, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, EEEEE)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'FF') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), FF, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, FF)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'ddd') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), ddd, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, ddd)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'DDDD') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), DDDD, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, DDDD)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'HHH') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), HHH, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, HHH)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'hhh') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), hhh, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, hhh)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'kkk') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), kkk, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, kkk)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'KKK') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), KKK, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, KKK)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'mmm') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), mmm, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, mmm)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'sss') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), sss, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, sss)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'SSSSSSSSSS') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), SSSSSSSSSS, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, SSSSSSSSSS)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'aa') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), aa, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, aa)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'V') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), V, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, V)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'zzzzz') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), zzzzz, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, zzzzz)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'XXXXXX') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), XXXXXX, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, XXXXXX)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'ZZZZZZ') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), ZZZZZZ, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, ZZZZZZ)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'OO') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), OO, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, OO)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'xxxxxx') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), xxxxxx, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, xxxxxx)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'A') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), A, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, A)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'n') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), n, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, n)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'N') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), N, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, N)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'p') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), p, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, p)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'Y') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), Y, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, Y)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'w') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), w, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, w)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'W') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), W, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, W)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'u') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), u, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, u)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'e') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), e, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, e)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'c') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), c, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, c)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'B') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), B, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, B)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'C') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), C, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, C)#x] ++- OneRowRelation + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'I') +-- !query analysis +Project [date_format(cast(2018-11-17 13:33:33.333 as timestamp), I, Some(America/Los_Angeles)) AS date_format(2018-11-17 13:33:33.333, I)#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting-legacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting-legacy.sql.out new file mode 100644 index 0000000000000..376f007233552 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting-legacy.sql.out @@ -0,0 +1,376 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view v as select col from values + (timestamp '1582-06-01 11:33:33.123UTC+080000'), + (timestamp '1970-01-01 00:00:00.000Europe/Paris'), + (timestamp '1970-12-31 23:59:59.999Asia/Srednekolymsk'), + (timestamp '1996-04-01 00:33:33.123Australia/Darwin'), + (timestamp '2018-11-17 13:33:33.123Z'), + (timestamp '2020-01-01 01:33:33.123Asia/Shanghai'), + (timestamp '2100-01-01 01:33:33.123America/Los_Angeles') t(col) +-- !query analysis +CreateViewCommand `v`, select col from values + (timestamp '1582-06-01 11:33:33.123UTC+080000'), + (timestamp '1970-01-01 00:00:00.000Europe/Paris'), + (timestamp '1970-12-31 23:59:59.999Asia/Srednekolymsk'), + (timestamp '1996-04-01 00:33:33.123Australia/Darwin'), + (timestamp '2018-11-17 13:33:33.123Z'), + (timestamp '2020-01-01 01:33:33.123Asia/Shanghai'), + (timestamp '2100-01-01 01:33:33.123America/Los_Angeles') t(col), false, false, LocalTempView, true + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'G GG GGG GGGG') from v +-- !query analysis +Project [col#x, date_format(col#x, G GG GGG GGGG, Some(America/Los_Angeles)) AS date_format(col, G GG GGG GGGG)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy') from v +-- !query analysis +Project [col#x, date_format(col#x, y yy yyy yyyy yyyyy yyyyyy, Some(America/Los_Angeles)) AS date_format(col, y yy yyy yyyy yyyyy yyyyyy)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'q qq') from v +-- !query analysis +Project [col#x, date_format(col#x, q qq, Some(America/Los_Angeles)) AS date_format(col, q qq)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'Q QQ QQQ QQQQ') from v +-- !query analysis +Project [col#x, date_format(col#x, Q QQ QQQ QQQQ, Some(America/Los_Angeles)) AS date_format(col, Q QQ QQQ QQQQ)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'M MM MMM MMMM') from v +-- !query analysis +Project [col#x, date_format(col#x, M MM MMM MMMM, Some(America/Los_Angeles)) AS date_format(col, M MM MMM MMMM)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'L LL') from v +-- !query analysis +Project [col#x, date_format(col#x, L LL, Some(America/Los_Angeles)) AS date_format(col, L LL)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'E EE EEE EEEE') from v +-- !query analysis +Project [col#x, date_format(col#x, E EE EEE EEEE, Some(America/Los_Angeles)) AS date_format(col, E EE EEE EEEE)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'F') from v +-- !query analysis +Project [col#x, date_format(col#x, F, Some(America/Los_Angeles)) AS date_format(col, F)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'd dd') from v +-- !query analysis +Project [col#x, date_format(col#x, d dd, Some(America/Los_Angeles)) AS date_format(col, d dd)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'DD') from v where col = timestamp '2100-01-01 01:33:33.123America/Los_Angeles' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select col, date_format(col, 'D DDD') from v +-- !query analysis +Project [col#x, date_format(col#x, D DDD, Some(America/Los_Angeles)) AS date_format(col, D DDD)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'H HH') from v +-- !query analysis +Project [col#x, date_format(col#x, H HH, Some(America/Los_Angeles)) AS date_format(col, H HH)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'h hh') from v +-- !query analysis +Project [col#x, date_format(col#x, h hh, Some(America/Los_Angeles)) AS date_format(col, h hh)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'k kk') from v +-- !query analysis +Project [col#x, date_format(col#x, k kk, Some(America/Los_Angeles)) AS date_format(col, k kk)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'K KK') from v +-- !query analysis +Project [col#x, date_format(col#x, K KK, Some(America/Los_Angeles)) AS date_format(col, K KK)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'm mm') from v +-- !query analysis +Project [col#x, date_format(col#x, m mm, Some(America/Los_Angeles)) AS date_format(col, m mm)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 's ss') from v +-- !query analysis +Project [col#x, date_format(col#x, s ss, Some(America/Los_Angeles)) AS date_format(col, s ss)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS') from v +-- !query analysis +Project [col#x, date_format(col#x, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS, Some(America/Los_Angeles)) AS date_format(col, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'a') from v +-- !query analysis +Project [col#x, date_format(col#x, a, Some(America/Los_Angeles)) AS date_format(col, a)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'VV') from v +-- !query analysis +Project [col#x, date_format(col#x, VV, Some(America/Los_Angeles)) AS date_format(col, VV)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'z zz zzz zzzz') from v +-- !query analysis +Project [col#x, date_format(col#x, z zz zzz zzzz, Some(America/Los_Angeles)) AS date_format(col, z zz zzz zzzz)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'X XX XXX') from v +-- !query analysis +Project [col#x, date_format(col#x, X XX XXX, Some(America/Los_Angeles)) AS date_format(col, X XX XXX)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'XXXX XXXXX') from v +-- !query analysis +Project [col#x, date_format(col#x, XXXX XXXXX, Some(America/Los_Angeles)) AS date_format(col, XXXX XXXXX)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'Z ZZ ZZZ ZZZZ ZZZZZ') from v +-- !query analysis +Project [col#x, date_format(col#x, Z ZZ ZZZ ZZZZ ZZZZZ, Some(America/Los_Angeles)) AS date_format(col, Z ZZ ZZZ ZZZZ ZZZZZ)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'O OOOO') from v +-- !query analysis +Project [col#x, date_format(col#x, O OOOO, Some(America/Los_Angeles)) AS date_format(col, O OOOO)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'x xx xxx xxxx xxxx xxxxx') from v +-- !query analysis +Project [col#x, date_format(col#x, x xx xxx xxxx xxxx xxxxx, Some(America/Los_Angeles)) AS date_format(col, x xx xxx xxxx xxxx xxxxx)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, '[yyyy-MM-dd HH:mm:ss]') from v +-- !query analysis +Project [col#x, date_format(col#x, [yyyy-MM-dd HH:mm:ss], Some(America/Los_Angeles)) AS date_format(col, [yyyy-MM-dd HH:mm:ss])#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'") from v +-- !query analysis +Project [col#x, date_format(col#x, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV', Some(America/Los_Angeles)) AS date_format(col, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV')#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, "''") from v +-- !query analysis +Project [col#x, date_format(col#x, '', Some(America/Los_Angeles)) AS date_format(col, '')#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, '') from v +-- !query analysis +Project [col#x, date_format(col#x, , Some(America/Los_Angeles)) AS date_format(col, )#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting.sql.out new file mode 100644 index 0000000000000..376f007233552 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-formatting.sql.out @@ -0,0 +1,376 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view v as select col from values + (timestamp '1582-06-01 11:33:33.123UTC+080000'), + (timestamp '1970-01-01 00:00:00.000Europe/Paris'), + (timestamp '1970-12-31 23:59:59.999Asia/Srednekolymsk'), + (timestamp '1996-04-01 00:33:33.123Australia/Darwin'), + (timestamp '2018-11-17 13:33:33.123Z'), + (timestamp '2020-01-01 01:33:33.123Asia/Shanghai'), + (timestamp '2100-01-01 01:33:33.123America/Los_Angeles') t(col) +-- !query analysis +CreateViewCommand `v`, select col from values + (timestamp '1582-06-01 11:33:33.123UTC+080000'), + (timestamp '1970-01-01 00:00:00.000Europe/Paris'), + (timestamp '1970-12-31 23:59:59.999Asia/Srednekolymsk'), + (timestamp '1996-04-01 00:33:33.123Australia/Darwin'), + (timestamp '2018-11-17 13:33:33.123Z'), + (timestamp '2020-01-01 01:33:33.123Asia/Shanghai'), + (timestamp '2100-01-01 01:33:33.123America/Los_Angeles') t(col), false, false, LocalTempView, true + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'G GG GGG GGGG') from v +-- !query analysis +Project [col#x, date_format(col#x, G GG GGG GGGG, Some(America/Los_Angeles)) AS date_format(col, G GG GGG GGGG)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy') from v +-- !query analysis +Project [col#x, date_format(col#x, y yy yyy yyyy yyyyy yyyyyy, Some(America/Los_Angeles)) AS date_format(col, y yy yyy yyyy yyyyy yyyyyy)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'q qq') from v +-- !query analysis +Project [col#x, date_format(col#x, q qq, Some(America/Los_Angeles)) AS date_format(col, q qq)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'Q QQ QQQ QQQQ') from v +-- !query analysis +Project [col#x, date_format(col#x, Q QQ QQQ QQQQ, Some(America/Los_Angeles)) AS date_format(col, Q QQ QQQ QQQQ)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'M MM MMM MMMM') from v +-- !query analysis +Project [col#x, date_format(col#x, M MM MMM MMMM, Some(America/Los_Angeles)) AS date_format(col, M MM MMM MMMM)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'L LL') from v +-- !query analysis +Project [col#x, date_format(col#x, L LL, Some(America/Los_Angeles)) AS date_format(col, L LL)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'E EE EEE EEEE') from v +-- !query analysis +Project [col#x, date_format(col#x, E EE EEE EEEE, Some(America/Los_Angeles)) AS date_format(col, E EE EEE EEEE)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'F') from v +-- !query analysis +Project [col#x, date_format(col#x, F, Some(America/Los_Angeles)) AS date_format(col, F)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'd dd') from v +-- !query analysis +Project [col#x, date_format(col#x, d dd, Some(America/Los_Angeles)) AS date_format(col, d dd)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'DD') from v where col = timestamp '2100-01-01 01:33:33.123America/Los_Angeles' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select col, date_format(col, 'D DDD') from v +-- !query analysis +Project [col#x, date_format(col#x, D DDD, Some(America/Los_Angeles)) AS date_format(col, D DDD)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'H HH') from v +-- !query analysis +Project [col#x, date_format(col#x, H HH, Some(America/Los_Angeles)) AS date_format(col, H HH)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'h hh') from v +-- !query analysis +Project [col#x, date_format(col#x, h hh, Some(America/Los_Angeles)) AS date_format(col, h hh)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'k kk') from v +-- !query analysis +Project [col#x, date_format(col#x, k kk, Some(America/Los_Angeles)) AS date_format(col, k kk)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'K KK') from v +-- !query analysis +Project [col#x, date_format(col#x, K KK, Some(America/Los_Angeles)) AS date_format(col, K KK)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'm mm') from v +-- !query analysis +Project [col#x, date_format(col#x, m mm, Some(America/Los_Angeles)) AS date_format(col, m mm)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 's ss') from v +-- !query analysis +Project [col#x, date_format(col#x, s ss, Some(America/Los_Angeles)) AS date_format(col, s ss)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS') from v +-- !query analysis +Project [col#x, date_format(col#x, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS, Some(America/Los_Angeles)) AS date_format(col, S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'a') from v +-- !query analysis +Project [col#x, date_format(col#x, a, Some(America/Los_Angeles)) AS date_format(col, a)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'VV') from v +-- !query analysis +Project [col#x, date_format(col#x, VV, Some(America/Los_Angeles)) AS date_format(col, VV)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'z zz zzz zzzz') from v +-- !query analysis +Project [col#x, date_format(col#x, z zz zzz zzzz, Some(America/Los_Angeles)) AS date_format(col, z zz zzz zzzz)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'X XX XXX') from v +-- !query analysis +Project [col#x, date_format(col#x, X XX XXX, Some(America/Los_Angeles)) AS date_format(col, X XX XXX)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'XXXX XXXXX') from v +-- !query analysis +Project [col#x, date_format(col#x, XXXX XXXXX, Some(America/Los_Angeles)) AS date_format(col, XXXX XXXXX)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'Z ZZ ZZZ ZZZZ ZZZZZ') from v +-- !query analysis +Project [col#x, date_format(col#x, Z ZZ ZZZ ZZZZ ZZZZZ, Some(America/Los_Angeles)) AS date_format(col, Z ZZ ZZZ ZZZZ ZZZZZ)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'O OOOO') from v +-- !query analysis +Project [col#x, date_format(col#x, O OOOO, Some(America/Los_Angeles)) AS date_format(col, O OOOO)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, 'x xx xxx xxxx xxxx xxxxx') from v +-- !query analysis +Project [col#x, date_format(col#x, x xx xxx xxxx xxxx xxxxx, Some(America/Los_Angeles)) AS date_format(col, x xx xxx xxxx xxxx xxxxx)#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, '[yyyy-MM-dd HH:mm:ss]') from v +-- !query analysis +Project [col#x, date_format(col#x, [yyyy-MM-dd HH:mm:ss], Some(America/Los_Angeles)) AS date_format(col, [yyyy-MM-dd HH:mm:ss])#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'") from v +-- !query analysis +Project [col#x, date_format(col#x, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV', Some(America/Los_Angeles)) AS date_format(col, 姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV')#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, "''") from v +-- !query analysis +Project [col#x, date_format(col#x, '', Some(America/Los_Angeles)) AS date_format(col, '')#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] + + +-- !query +select col, date_format(col, '') from v +-- !query analysis +Project [col#x, date_format(col#x, , Some(America/Los_Angeles)) AS date_format(col, )#x] ++- SubqueryAlias v + +- View (`v`, [col#x]) + +- Project [cast(col#x as timestamp) AS col#x] + +- Project [col#x] + +- SubqueryAlias t + +- LocalRelation [col#x] 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 new file mode 100644 index 0000000000000..9eb83778af839 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out @@ -0,0 +1,1804 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view date_view as select '2011-11-11' date_str, '1' int_str +-- !query analysis +CreateViewCommand `date_view`, select '2011-11-11' date_str, '1' int_str, false, false, LocalTempView, true + +- Project [2011-11-11 AS date_str#x, 1 AS int_str#x] + +- OneRowRelation + + +-- !query +select date '2019-01-01\t' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2020-01-01中文' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2020-01-01中文'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "date '2020-01-01中文'" + } ] +} + + +-- !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] ++- OneRowRelation + + +-- !query +select make_date(2000, 13, 1) +-- !query analysis +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, false) AS make_date(2000, 1, 33)#x] ++- OneRowRelation + + +-- !query +select date'015' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'015'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "date'015'" + } ] +} + + +-- !query +select date'2021-4294967297-11' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-4294967297-11'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "date'2021-4294967297-11'" + } ] +} + + +-- !query +select current_date = current_date +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select current_date() = current_date() +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select curdate(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "0", + "functionName" : "`curdate`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "curdate(1)" + } ] +} + + +-- !query +select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null) +-- !query analysis +Project [date_from_unix_date(0) AS date_from_unix_date(0)#x, date_from_unix_date(1000) AS date_from_unix_date(1000)#x, date_from_unix_date(cast(null as int)) AS date_from_unix_date(NULL)#x] ++- OneRowRelation + + +-- !query +select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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)) AS to_date(NULL)#x, to_date(2016-12-31, None, Some(America/Los_Angeles)) AS to_date(2016-12-31)#x, to_date(2016-12-31, Some(yyyy-MM-dd), Some(America/Los_Angeles)) 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)) 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)) AS to_date(02-29, MM-dd)#x] ++- OneRowRelation + + +-- !query +select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), + dayofweek('1582-10-15 13:10:15'), dayofweek(timestamp_ltz'1582-10-15 13:10:15'), dayofweek(timestamp_ntz'1582-10-15 13:10:15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), + weekday('1582-10-15 13:10:15'), weekday(timestamp_ltz'1582-10-15 13:10:15'), weekday(timestamp_ntz'1582-10-15 13:10:15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select year('1500-01-01'), year('1582-10-15 13:10:15'), year(timestamp_ltz'1582-10-15 13:10:15'), year(timestamp_ntz'1582-10-15 13:10:15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select month('1500-01-01'), month('1582-10-15 13:10:15'), month(timestamp_ltz'1582-10-15 13:10:15'), month(timestamp_ntz'1582-10-15 13:10:15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(timestamp_ltz'1582-10-15 13:10:15'), dayOfYear(timestamp_ntz'1582-10-15 13:10:15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon") +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- OneRowRelation + + +-- !query +select next_day("xx", "Mon") +-- !query analysis +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, false) 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] ++- OneRowRelation + + +-- !query +select date_add(date'2011-11-11', 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- OneRowRelation + + +-- !query +select date_add('2011-11-11', 1Y) +-- !query analysis +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', 1S) +-- !query analysis +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', 1L) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "date_add('2011-11-11', 1L)" + } ] +} + + +-- !query +select date_add('2011-11-11', 1.0) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_add('2011-11-11', 1.0)" + } ] +} + + +-- !query +select date_add('2011-11-11', 1E1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_add('2011-11-11', 1E1)" + } ] +} + + +-- !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] ++- 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" + } +} + + +-- !query +select date_add(null, 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_add(date'2011-11-11', null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_add(timestamp_ltz'2011-11-11 12:12:12', 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_add(timestamp_ntz'2011-11-11 12:12:12', 1) +-- !query analysis +Project [date_add(cast(2011-11-11 12:12:12 as date), 1) AS date_add(TIMESTAMP_NTZ '2011-11-11 12:12:12', 1)#x] ++- OneRowRelation + + +-- !query +select date_sub(date'2011-11-11', 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_sub('2011-11-11', 1) +-- !query analysis +Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x] ++- OneRowRelation + + +-- !query +select date_sub('2011-11-11', 1Y) +-- !query analysis +Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x] ++- OneRowRelation + + +-- !query +select date_sub('2011-11-11', 1S) +-- !query analysis +Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x] ++- OneRowRelation + + +-- !query +select date_sub('2011-11-11', 1L) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "date_sub('2011-11-11', 1L)" + } ] +} + + +-- !query +select date_sub('2011-11-11', 1.0) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_sub('2011-11-11', 1.0)" + } ] +} + + +-- !query +select date_sub('2011-11-11', 1E1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_sub('2011-11-11', 1E1)" + } ] +} + + +-- !query +select date_sub(date'2011-11-11', '1') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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" + } +} + + +-- !query +select date_sub(null, 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_sub(date'2011-11-11', null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_sub(timestamp_ltz'2011-11-11 12:12:12', 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_sub(timestamp_ntz'2011-11-11 12:12:12', 1) +-- !query analysis +Project [date_sub(cast(2011-11-11 12:12:12 as date), 1) AS date_sub(TIMESTAMP_NTZ '2011-11-11 12:12:12', 1)#x] ++- OneRowRelation + + +-- !query +select date_add('2011-11-11', int_str) from date_view +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"int_str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "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 +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"int_str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "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 +select date_add(date_str, 1) from date_view +-- !query analysis +Project [date_add(cast(date_str#x as date), 1) AS date_add(date_str, 1)#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(date_str, 1) from date_view +-- !query analysis +Project [date_sub(cast(date_str#x as date), 1) AS date_sub(date_str, 1)#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 '2011-11-11' + 1E1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "date '2011-11-11' + 1E1" + } ] +} + + +-- !query +select date '2001-09-28' + 7Y +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select 7S + date '2001-09-28' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2001-10-01' - 7 +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2001-10-01' - date '2001-09-28' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2001-10-01' - '2001-09-28' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2001-09-28\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "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 +select '2001-10-01' - date '2001-09-28' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2001-09-28' - null +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select null - date '2019-10-06' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date_str - date '2001-09-28' from date_view +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2001-09-28' - date_str from date_view +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"date_str\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "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 +select date'2011-11-11' + '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "date'2011-11-11' + '1'" + } ] +} + + +-- !query +select '1' + date'2011-11-11' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' + date'2011-11-11'" + } ] +} + + +-- !query +select date'2011-11-11' + null +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select null + date'2011-11-11' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2012-01-01' - interval '2-2' year to month, + date '2011-11-11' - interval '2' day, + date '2012-01-01' + interval '-2-2' year to month, + date '2011-11-11' + interval '-2' month, + - interval '2-2' year to month + date '2012-01-01', + interval '-2' day + date '2011-11-11' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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)) AS to_date(26/October/2015, dd/MMMMM/yyyy)#x] ++- OneRowRelation + + +-- !query +select from_json('{"d":"26/October/2015"}', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_json(StructField(d,DateType,true), (dateFormat,dd/MMMMM/yyyy), {"d":"26/October/2015"}, Some(America/Los_Angeles)) AS from_json({"d":"26/October/2015"})#x] ++- OneRowRelation + + +-- !query +select from_csv('26/October/2015', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_csv(StructField(d,DateType,true), (dateFormat,dd/MMMMM/yyyy), 26/October/2015, Some(America/Los_Angeles), None) AS from_csv(26/October/2015)#x] ++- OneRowRelation + + +-- !query +select dateadd(MICROSECOND, 1001, timestamp'2022-02-25 01:02:03.123') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(MILLISECOND, -1, timestamp'2022-02-25 01:02:03.456') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(SECOND, 58, timestamp'2022-02-25 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(MINUTE, -100, date'2022-02-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(HOUR, -1, timestamp'2022-02-25 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(DAY, 367, date'2022-02-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(WEEK, -4, timestamp'2022-02-25 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(MONTH, -1, timestamp'2022-02-25 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(QUARTER, 5, date'2022-02-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dateadd(YEAR, 1, date'2022-02-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(MICROSECOND, timestamp'2022-02-25 01:02:03.123', timestamp'2022-02-25 01:02:03.124001') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(MILLISECOND, timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(SECOND, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 01:03:01') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(MINUTE, date'2022-02-25', timestamp'2022-02-24 22:20:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(HOUR, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 00:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(DAY, date'2022-02-25', timestamp'2023-02-27 00:00:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(WEEK, timestamp'2022-02-25 01:02:03', timestamp'2022-01-28 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(MONTH, timestamp'2022-02-25 01:02:03', timestamp'2022-01-25 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(QUARTER, date'2022-02-25', date'2023-05-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select datediff(YEAR, date'2022-02-25', date'2023-02-25') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp '2019-01-01\t' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp '2019-01-01中文' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2019-01-01中文'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} + + +-- !query +select timestamp'4294967297' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'4294967297'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} + + +-- !query +select timestamp'2021-01-01T12:30:4294967297.123456' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-01-01T12:30:4294967297.123456'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} + + +-- !query +select current_timestamp = current_timestamp +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select current_timestamp() = current_timestamp() +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select localtimestamp() = localtimestamp() +-- !query analysis +Project [(localtimestamp(Some(America/Los_Angeles)) = localtimestamp(Some(America/Los_Angeles))) AS (localtimestamp() = localtimestamp())#x] ++- OneRowRelation + + +-- !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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) +-- !query analysis +Project [timestamp_seconds(1230219000) AS timestamp_seconds(1230219000)#x, timestamp_seconds(-1230219000) AS timestamp_seconds(-1230219000)#x, timestamp_seconds(null) AS timestamp_seconds(NULL)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)) +-- !query analysis +Project [timestamp_seconds(1.23) AS timestamp_seconds(1.23)#x, timestamp_seconds(1.23) AS timestamp_seconds(1.23)#x, timestamp_seconds(cast(1.23 as float)) AS timestamp_seconds(1.23)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null) +-- !query analysis +Project [timestamp_millis(1230219000123) AS timestamp_millis(1230219000123)#x, timestamp_millis(-1230219000123) AS timestamp_millis(-1230219000123)#x, timestamp_millis(null) AS timestamp_millis(NULL)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null) +-- !query analysis +Project [timestamp_micros(1230219000123123) AS timestamp_micros(1230219000123123)#x, timestamp_micros(-1230219000123123) AS timestamp_micros(-1230219000123123)#x, timestamp_micros(null) AS timestamp_micros(NULL)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(1230219000123123) +-- !query analysis +Project [timestamp_seconds(1230219000123123) AS timestamp_seconds(1230219000123123)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(-1230219000123123) +-- !query analysis +Project [timestamp_seconds(-1230219000123123) AS timestamp_seconds(-1230219000123123)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MILLIS(92233720368547758) +-- !query analysis +Project [timestamp_millis(92233720368547758) AS timestamp_millis(92233720368547758)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MILLIS(-92233720368547758) +-- !query analysis +Project [timestamp_millis(-92233720368547758) AS timestamp_millis(-92233720368547758)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(0.1234567) +-- !query analysis +Project [timestamp_seconds(0.1234567) AS timestamp_seconds(0.1234567)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)) +-- !query analysis +Project [timestamp_seconds(0.1234567) AS timestamp_seconds(0.1234567)#x, timestamp_seconds(cast(0.1234567 as float)) AS timestamp_seconds(0.1234567)#x] ++- OneRowRelation + + +-- !query +create temporary view ttf1 as select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`) +-- !query analysis +CreateViewCommand `ttf1`, select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`), false, false, LocalTempView, true + +- Project [current_date#x, current_timestamp#x] + +- SubqueryAlias ttf1 + +- LocalRelation [current_date#x, current_timestamp#x] + + +-- !query +select typeof(current_date), typeof(current_timestamp) from ttf1 +-- !query analysis +Project [typeof(current_date#x) AS typeof(current_date)#x, typeof(current_timestamp#x) AS typeof(current_timestamp)#x] ++- SubqueryAlias ttf1 + +- View (`ttf1`, [current_date#x,current_timestamp#x]) + +- Project [cast(current_date#x as int) AS current_date#x, cast(current_timestamp#x as int) AS current_timestamp#x] + +- Project [current_date#x, current_timestamp#x] + +- SubqueryAlias ttf1 + +- LocalRelation [current_date#x, current_timestamp#x] + + +-- !query +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b) +-- !query analysis +CreateViewCommand `ttf2`, select * from values + (1, 2), + (2, 3) + as ttf2(a, b), false, false, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias ttf2 + +- LocalRelation [a#x, b#x] + + +-- !query +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2 +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select a, b from ttf2 order by a, current_date +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- OneRowRelation + + +-- !query +select to_timestamp(1) +-- !query analysis +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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), false) AS to_timestamp(2019 10:10:10, yyyy hh:mm:ss)#x] ++- OneRowRelation + + +-- !query +select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:10\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "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 +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:11\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "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 +select timestamp'2011-11-11 11:11:11' - null +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select null - timestamp'2011-11-11 11:11:11' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +create temporary view ts_view as select '2011-11-11 11:11:11' str +-- !query analysis +CreateViewCommand `ts_view`, select '2011-11-11 11:11:11' str, false, false, LocalTempView, true + +- Project [2011-11-11 11:11:11 AS str#x] + +- OneRowRelation + + +-- !query +select str - timestamp'2011-11-11 11:11:11' from ts_view +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "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 +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "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 +select timestamp'2011-11-11 11:11:11' + '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" + } ] +} + + +-- !query +select '1' + timestamp'2011-11-11 11:11:11' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + null +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "timestamp'2011-11-11 11:11:11' + null" + } ] +} + + +-- !query +select null + timestamp'2011-11-11 11:11:11' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "null + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + interval '2' day, + timestamp'2011-11-11 11:11:11' - interval '2-2' year to month, + timestamp'2011-11-11 11:11:11' + interval '-2' second, + timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second, + - interval 2 years + timestamp'2011-11-11 11:11:11', + interval '1 12' day to hour + timestamp'2011-11-11 11:11:11' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2012-01-01' - interval 3 hours, + date '2012-01-01' + interval '12:12:12' hour to second, + interval '2' minute + date '2012-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- 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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_json(StructField(t,TimestampType,true), (timestampFormat,dd/MMMMM/yyyy), {"t":"26/October/2015"}, Some(America/Los_Angeles)) AS from_json({"t":"26/October/2015"})#x] ++- OneRowRelation + + +-- !query +select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_csv(StructField(t,TimestampType,true), (timestampFormat,dd/MMMMM/yyyy), 26/October/2015, Some(America/Los_Angeles), None) AS from_csv(26/October/2015)#x] ++- OneRowRelation + + +-- !query +select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampadd(YEAR, 1, date'2022-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampadd(SECOND, -1, date'2022-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] 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 new file mode 100644 index 0000000000000..783a4b2156f4e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-invalid.sql.out @@ -0,0 +1,195 @@ +-- Automatically generated by SQLQueryTestSuite +-- !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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD')) +-- !query analysis +Project [from_csv(StructField(date,DateType,true), (dateFormat,yyyy-DDD), 2018-366, Some(America/Los_Angeles), None) AS from_csv(2018-366)#x] ++- OneRowRelation + + +-- !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)) 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)) 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select cast("Unparseable" as timestamp) +-- !query analysis +Project [cast(Unparseable as timestamp) AS CAST(Unparseable AS TIMESTAMP)#x] ++- OneRowRelation + + +-- !query +select cast("Unparseable" as date) +-- !query analysis +Project [cast(Unparseable as date) AS CAST(Unparseable AS DATE)#x] ++- 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 new file mode 100644 index 0000000000000..f24dee822749e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-legacy.sql.out @@ -0,0 +1,174 @@ +-- Automatically generated by SQLQueryTestSuite +-- !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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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 new file mode 100644 index 0000000000000..f24dee822749e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing.sql.out @@ -0,0 +1,174 @@ +-- Automatically generated by SQLQueryTestSuite +-- !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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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 new file mode 100644 index 0000000000000..01d1f2c40a4a6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-special.sql.out @@ -0,0 +1,25 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select date'999999-03-18', date'-0001-1-28', date'0015' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- OneRowRelation + + +-- !query +select timestamp'-1969-12-31 16:00:00', timestamp'-0015-03-18 16:00:00', timestamp'-000001', timestamp'99999-03-18T12:03:17' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- 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 new file mode 100644 index 0000000000000..31a46aa039927 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/decimalArithmeticOperations.sql.out @@ -0,0 +1,445 @@ +-- 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, 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 +CreateDataSourceTableCommand `spark_catalog`.`default`.`decimals_test`, false + + +-- !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 analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/decimals_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/decimals_test], Append, `spark_catalog`.`default`.`decimals_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/decimals_test), [id, a, b] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,18)) AS a#x, cast(col3#x as decimal(38,18)) AS b#x] + +- 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 +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 +Project [(123456789123456789.1234567890 * 1.123456789123456789) AS (123456789123456789.1234567890 * 1.123456789123456789)#x] ++- 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 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 +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 +Project [(123456789123456789.1234567890 * 1.123456789123456789) AS (123456789123456789.1234567890 * 1.123456789123456789)#x] ++- 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 1.0123456789012345678901234567890123456e36BD / 0.1 +-- !query analysis +Project [(1012345678901234567890123456789012345.6 / 0.1) AS (1012345678901234567890123456789012345.6 / 0.1)#x] ++- OneRowRelation + + +-- !query +select 1.0123456789012345678901234567890123456e35BD / 1.0 +-- !query analysis +Project [(101234567890123456789012345678901234.56 / 1.0) AS (101234567890123456789012345678901234.56 / 1.0)#x] ++- OneRowRelation + + +-- !query +select 1.0123456789012345678901234567890123456e34BD / 1.0 +-- !query analysis +Project [(10123456789012345678901234567890123.456 / 1.0) AS (10123456789012345678901234567890123.456 / 1.0)#x] ++- OneRowRelation + + +-- !query +select 1.0123456789012345678901234567890123456e33BD / 1.0 +-- !query analysis +Project [(1012345678901234567890123456789012.3456 / 1.0) AS (1012345678901234567890123456789012.3456 / 1.0)#x] ++- OneRowRelation + + +-- !query +select 1.0123456789012345678901234567890123456e32BD / 1.0 +-- !query analysis +Project [(101234567890123456789012345678901.23456 / 1.0) AS (101234567890123456789012345678901.23456 / 1.0)#x] ++- OneRowRelation + + +-- !query +select 1.0123456789012345678901234567890123456e31BD / 1.0 +-- !query analysis +Project [(10123456789012345678901234567890.123456 / 1.0) AS (10123456789012345678901234567890.123456 / 1.0)#x] ++- OneRowRelation + + +-- !query +select 1.0123456789012345678901234567890123456e31BD / 0.1 +-- !query analysis +Project [(10123456789012345678901234567890.123456 / 0.1) AS (10123456789012345678901234567890.123456 / 0.1)#x] ++- OneRowRelation + + +-- !query +select 1.0123456789012345678901234567890123456e31BD / 10.0 +-- !query analysis +Project [(10123456789012345678901234567890.123456 / 10.0) AS (10123456789012345678901234567890.123456 / 10.0)#x] ++- OneRowRelation + + +-- !query +drop table decimals_test +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`decimals_test`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe-part-after-analyze.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe-part-after-analyze.sql.out new file mode 100644 index 0000000000000..12e5edaddd21c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe-part-after-analyze.sql.out @@ -0,0 +1,102 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE t (key STRING, value STRING, ds STRING, hr INT) USING parquet + PARTITIONED BY (ds, hr) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +INSERT INTO TABLE t PARTITION (ds='2017-08-01', hr=10) +VALUES ('k1', 100), ('k2', 200), ('k3', 300) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, [ds=2017-08-01, hr=10], false, [ds#x, hr#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [key, value, ds, hr] ++- Project [key#x, value#x, cast(2017-08-01 as string) AS ds#x, cast(10 as int) AS hr#x] + +- Project [cast(col1#x as string) AS key#x, cast(col2#x as string) AS value#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO TABLE t PARTITION (ds='2017-08-01', hr=11) +VALUES ('k1', 101), ('k2', 201), ('k3', 301), ('k4', 401) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, [ds=2017-08-01, hr=11], false, [ds#x, hr#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [key, value, ds, hr] ++- Project [key#x, value#x, cast(2017-08-01 as string) AS ds#x, cast(11 as int) AS hr#x] + +- Project [cast(col1#x as string) AS key#x, cast(col2#x as string) AS value#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO TABLE t PARTITION (ds='2017-09-01', hr=5) +VALUES ('k1', 102), ('k2', 202) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, [ds=2017-09-01, hr=5], false, [ds#x, hr#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [key, value, ds, hr] ++- Project [key#x, value#x, cast(2017-09-01 as string) AS ds#x, cast(5 as int) AS hr#x] + +- Project [cast(col1#x as string) AS key#x, cast(col2#x as string) AS value#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, [ds=2017-08-01, hr=10], true, [col_name#x, data_type#x, comment#x] + + +-- !query +ANALYZE TABLE t PARTITION (ds='2017-08-01', hr=10) COMPUTE STATISTICS +-- !query analysis +AnalyzePartitionCommand `spark_catalog`.`default`.`t`, [ds=Some(2017-08-01), hr=Some(10)], false + + +-- !query +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, [ds=2017-08-01, hr=10], true, [col_name#x, data_type#x, comment#x] + + +-- !query +ANALYZE TABLE t PARTITION (ds='2017-08-01') COMPUTE STATISTICS +-- !query analysis +AnalyzePartitionCommand `spark_catalog`.`default`.`t`, [ds=Some(2017-08-01)], false + + +-- !query +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, [ds=2017-08-01, hr=10], true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11) +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, [ds=2017-08-01, hr=11], true, [col_name#x, data_type#x, comment#x] + + +-- !query +ANALYZE TABLE t PARTITION (ds, hr) COMPUTE STATISTICS +-- !query analysis +AnalyzePartitionCommand `spark_catalog`.`default`.`t`, [ds=None, hr=None], false + + +-- !query +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, [ds=2017-08-01, hr=10], true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11) +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, [ds=2017-08-01, hr=11], true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC EXTENDED t PARTITION (ds='2017-09-01', hr=5) +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, [ds=2017-09-01, hr=5], true, [col_name#x, data_type#x, comment#x] + + +-- !query +DROP TABLE t +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe-query.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe-query.sql.out new file mode 100644 index 0000000000000..c2a9f8e07bcc0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe-query.sql.out @@ -0,0 +1,140 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE table desc_temp1 (key int COMMENT 'column_comment', val string) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`desc_temp1`, false + + +-- !query +CREATE table desc_temp2 (key int, val string) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`desc_temp2`, false + + +-- !query +DESC SELECT key, key + 1 as plusone FROM desc_temp1 +-- !query analysis +DescribeQueryCommand SELECT key, key + 1 as plusone FROM desc_temp1 + + +-- !query +DESC QUERY SELECT * FROM desc_temp2 +-- !query analysis +DescribeQueryCommand SELECT * FROM desc_temp2 + + +-- !query +DESC SELECT key, COUNT(*) as count FROM desc_temp1 group by key +-- !query analysis +DescribeQueryCommand SELECT key, COUNT(*) as count FROM desc_temp1 group by key + + +-- !query +DESC SELECT 10.00D as col1 +-- !query analysis +DescribeQueryCommand SELECT 10.00D as col1 + + +-- !query +DESC QUERY SELECT key FROM desc_temp1 UNION ALL select CAST(1 AS DOUBLE) +-- !query analysis +DescribeQueryCommand SELECT key FROM desc_temp1 UNION ALL select CAST(1 AS DOUBLE) + + +-- !query +DESC QUERY VALUES(1.00D, 'hello') as tab1(col1, col2) +-- !query analysis +DescribeQueryCommand VALUES(1.00D, 'hello') as tab1(col1, col2) + + +-- !query +DESC QUERY FROM desc_temp1 a SELECT * +-- !query analysis +DescribeQueryCommand FROM desc_temp1 a SELECT * + + +-- !query +DESC WITH s AS (SELECT 'hello' as col1) SELECT * FROM s +-- !query analysis +DescribeQueryCommand WITH s AS (SELECT 'hello' as col1) SELECT * FROM s + + +-- !query +DESCRIBE QUERY WITH s AS (SELECT * from desc_temp1) SELECT * FROM s +-- !query analysis +DescribeQueryCommand WITH s AS (SELECT * from desc_temp1) SELECT * FROM s + + +-- !query +DESCRIBE SELECT * FROM (FROM desc_temp2 select * select *) +-- !query analysis +DescribeQueryCommand SELECT * FROM (FROM desc_temp2 select * select *) + + +-- !query +DESCRIBE INSERT INTO desc_temp1 values (1, 'val1') +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'desc_temp1'", + "hint" : "" + } +} + + +-- !query +DESCRIBE INSERT INTO desc_temp1 SELECT * FROM desc_temp2 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'desc_temp1'", + "hint" : "" + } +} + + +-- !query +DESCRIBE + FROM desc_temp1 a + insert into desc_temp1 select * + insert into desc_temp2 select * +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'insert'", + "hint" : "" + } +} + + +-- !query +EXPLAIN DESC QUERY SELECT * FROM desc_temp2 WHERE key > 0 +-- !query analysis +ExplainCommand DescribeQueryCommand SELECT * FROM desc_temp2 WHERE key > 0, SimpleMode + + +-- !query +EXPLAIN EXTENDED DESC WITH s AS (SELECT 'hello' as col1) SELECT * FROM s +-- !query analysis +ExplainCommand DescribeQueryCommand WITH s AS (SELECT 'hello' as col1) SELECT * FROM s, ExtendedMode + + +-- !query +DROP TABLE desc_temp1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`desc_temp1`, false, false, false + + +-- !query +DROP TABLE desc_temp2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`desc_temp2`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe-table-after-alter-table.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe-table-after-alter-table.sql.out new file mode 100644 index 0000000000000..a23e3596922b0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe-table-after-alter-table.sql.out @@ -0,0 +1,71 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE table_with_comment (a STRING, b INT, c STRING, d STRING) USING parquet COMMENT 'added' +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`table_with_comment`, false + + +-- !query +DESC FORMATTED table_with_comment +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`table_with_comment`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE table_with_comment SET TBLPROPERTIES("comment"= "modified comment", "type"= "parquet") +-- !query analysis +AlterTableSetPropertiesCommand `spark_catalog`.`default`.`table_with_comment`, [comment=modified comment, type=parquet], false + + +-- !query +DESC FORMATTED table_with_comment +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`table_with_comment`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DROP TABLE table_with_comment +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`table_with_comment`, false, false, false + + +-- !query +CREATE TABLE table_comment (a STRING, b INT) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`table_comment`, false + + +-- !query +DESC FORMATTED table_comment +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`table_comment`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE table_comment SET TBLPROPERTIES(comment = "added comment") +-- !query analysis +AlterTableSetPropertiesCommand `spark_catalog`.`default`.`table_comment`, [comment=added comment], false + + +-- !query +DESC formatted table_comment +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`table_comment`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE table_comment UNSET TBLPROPERTIES IF EXISTS ('comment') +-- !query analysis +AlterTableUnsetPropertiesCommand `spark_catalog`.`default`.`table_comment`, [comment], true, false + + +-- !query +DESC FORMATTED table_comment +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`table_comment`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DROP TABLE table_comment +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`table_comment`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out new file mode 100644 index 0000000000000..cd0a8585d740b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out @@ -0,0 +1,388 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet + OPTIONS (a '1', b '2', password 'password') + PARTITIONED BY (c, d) CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS + COMMENT 'table_comment' + TBLPROPERTIES (t 'test', password 'password') +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +CREATE TEMPORARY VIEW temp_v AS SELECT * FROM t +-- !query analysis +CreateViewCommand `temp_v`, SELECT * FROM t, false, false, LocalTempView, true + +- Project [a#x, b#x, c#x, d#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[a#x,b#x,c#x,d#x] parquet + + +-- !query +CREATE TEMPORARY VIEW temp_Data_Source_View + USING org.apache.spark.sql.sources.DDLScanSource + OPTIONS ( + From '1', + To '10', + Table 'test1') +-- !query analysis +CreateTempViewUsing [tableIdent:`temp_Data_Source_View` replace:false provider:org.apache.spark.sql.sources.DDLScanSource Map(From -> 1, To -> 10, Table -> test1) + + +-- !query +CREATE VIEW v AS SELECT * FROM t +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`v`, SELECT * FROM t, false, false, PersistedView, true + +- Project [a#x, b#x, c#x, d#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[a#x,b#x,c#x,d#x] parquet + + +-- !query +ALTER TABLE t SET TBLPROPERTIES (e = '3') +-- !query analysis +AlterTableSetPropertiesCommand `spark_catalog`.`default`.`t`, [e=3], false + + +-- !query +ALTER TABLE t ADD PARTITION (c='Us', d=1) +-- !query analysis +AlterTableAddPartitionCommand `spark_catalog`.`default`.`t`, [(Map(c -> Us, d -> 1),None)], false + + +-- !query +DESCRIBE t +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC default.t +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE t +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC FORMATTED t +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC EXTENDED t +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE t UNSET TBLPROPERTIES (e) +-- !query analysis +AlterTableUnsetPropertiesCommand `spark_catalog`.`default`.`t`, [e], false, false + + +-- !query +DESC EXTENDED t +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE t UNSET TBLPROPERTIES (comment) +-- !query analysis +AlterTableUnsetPropertiesCommand `spark_catalog`.`default`.`t`, [comment], false, false + + +-- !query +DESC EXTENDED t +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC t PARTITION (c='Us', d=1) +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, [c=Us, d=1], false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC EXTENDED t PARTITION (c='Us', d=1) +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, [c=Us, d=1], true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC FORMATTED t PARTITION (c='Us', d=1) +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, [c=Us, d=1], true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC EXTENDED t PARTITION (C='Us', D=1) +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`t`, [C=Us, D=1], true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC t PARTITION (c='Us', d=2) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException +{ + "errorClass" : "PARTITIONS_NOT_FOUND", + "sqlState" : "428FT", + "messageParameters" : { + "partitionList" : "PARTITION (`c` = Us, `d` = 2)", + "tableName" : "`default`.`t`" + } +} + + +-- !query +DESC t PARTITION (c='Us') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1232", + "messageParameters" : { + "partitionColumnNames" : "c, d", + "specKeys" : "c", + "tableName" : "`spark_catalog`.`default`.`t`" + } +} + + +-- !query +DESC t PARTITION (c='Us', d) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX", + "sqlState" : "42000", + "messageParameters" : { + "inputString" : "PARTITION specification is incomplete: `d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 28, + "fragment" : "DESC t PARTITION (c='Us', d)" + } ] +} + + +-- !query +DESC temp_v +-- !query analysis +DescribeTableCommand `temp_v`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE temp_v +-- !query analysis +DescribeTableCommand `temp_v`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC FORMATTED temp_v +-- !query analysis +DescribeTableCommand `temp_v`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC EXTENDED temp_v +-- !query analysis +DescribeTableCommand `temp_v`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC temp_Data_Source_View +-- !query analysis +DescribeTableCommand `temp_Data_Source_View`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC temp_v PARTITION (c='Us', d=1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "FORBIDDEN_OPERATION", + "sqlState" : "42809", + "messageParameters" : { + "objectName" : "`temp_v`", + "objectType" : "TEMPORARY VIEW", + "statement" : "DESC PARTITION" + } +} + + +-- !query +DESC v +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`v`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE v +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`v`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC FORMATTED v +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC EXTENDED v +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`v`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC v PARTITION (c='Us', d=1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "FORBIDDEN_OPERATION", + "sqlState" : "42809", + "messageParameters" : { + "objectName" : "`v`", + "objectType" : "VIEW", + "statement" : "DESC PARTITION" + } +} + + +-- !query +EXPLAIN DESC t +-- !query analysis +ExplainCommand 'DescribeRelation false, [col_name#x, data_type#x, comment#x], SimpleMode + + +-- !query +EXPLAIN DESC EXTENDED t +-- !query analysis +ExplainCommand 'DescribeRelation true, [col_name#x, data_type#x, comment#x], SimpleMode + + +-- !query +EXPLAIN EXTENDED DESC t +-- !query analysis +ExplainCommand 'DescribeRelation false, [col_name#x, data_type#x, comment#x], ExtendedMode + + +-- !query +EXPLAIN DESCRIBE t b +-- !query analysis +ExplainCommand 'DescribeColumn 'b, false, [info_name#x, info_value#x], SimpleMode + + +-- !query +EXPLAIN DESCRIBE t PARTITION (c='Us', d=2) +-- !query analysis +ExplainCommand 'DescribeRelation [c=Us, d=2], false, [col_name#x, data_type#x, comment#x], SimpleMode + + +-- !query +DROP TABLE t +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t`, false, false, false + + +-- !query +DROP VIEW temp_v +-- !query analysis +DropTempViewCommand temp_v + + +-- !query +DROP VIEW temp_Data_Source_View +-- !query analysis +DropTempViewCommand temp_Data_Source_View + + +-- !query +DROP VIEW v +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`v`, false, true, false + + +-- !query +CREATE TABLE d (a STRING DEFAULT 'default-value', b INT DEFAULT 42) USING parquet COMMENT 'table_comment' +-- !query analysis +org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +{ + "errorClass" : "TABLE_OR_VIEW_ALREADY_EXISTS", + "sqlState" : "42P07", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`d`" + } +} + + +-- !query +DESC d +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`d`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC EXTENDED d +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`d`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE EXTENDED d +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`d`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC FORMATTED d +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`d`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE TABLE e (a STRING DEFAULT CONCAT('a\n b\n ', 'c\n d'), b INT DEFAULT 42) USING parquet COMMENT 'table_comment' +-- !query analysis +org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +{ + "errorClass" : "TABLE_OR_VIEW_ALREADY_EXISTS", + "sqlState" : "42P07", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`e`" + } +} + + +-- !query +DESC e +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`e`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC EXTENDED e +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`e`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE EXTENDED e +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`e`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC FORMATTED e +-- !query analysis +DescribeTableCommand `spark_catalog`.`default`.`e`, true, [col_name#x, data_type#x, comment#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers.sql.out new file mode 100644 index 0000000000000..501eec2c963ac --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers.sql.out @@ -0,0 +1,393 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT 1 FROM "not_exist" +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +USE SCHEMA "not_exist" +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +ALTER TABLE "not_exist" ADD COLUMN not_exist int +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +ALTER TABLE not_exist ADD COLUMN "not_exist" int +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT 1 AS "not_exist" FROM not_exist +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT 1 FROM not_exist AS X("hello") +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"hello\"'", + "hint" : "" + } +} + + +-- !query +SELECT "not_exist"() +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT "not_exist".not_exist() +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT 1 FROM `hello` +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`hello`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 21, + "fragment" : "`hello`" + } ] +} + + +-- !query +USE SCHEMA `not_exist` +-- !query analysis +org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +{ + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42704", + "messageParameters" : { + "schemaName" : "`not_exist`" + } +} + + +-- !query +ALTER TABLE `not_exist` ADD COLUMN not_exist int +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "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.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "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.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 40, + "fragment" : "`not_exist`" + } ] +} + + +-- !query +SELECT 1 FROM not_exist AS X(`hello`) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 37, + "fragment" : "not_exist AS X(`hello`)" + } ] +} + + +-- !query +SELECT `not_exist`() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "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.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "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()" + } ] +} + + +-- !query +SELECT "hello" +-- !query analysis +Project [hello AS hello#x] ++- OneRowRelation + + +-- !query +CREATE TEMPORARY VIEW v(c1 COMMENT "hello") AS SELECT 1 +-- !query analysis +CreateViewCommand `v`, [(c1,Some(hello))], SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +DROP VIEW v +-- !query analysis +DropTempViewCommand v + + +-- !query +SELECT INTERVAL "1" YEAR +-- !query analysis +Project [INTERVAL '1' YEAR AS INTERVAL '1' YEAR#x] ++- OneRowRelation + + +-- !query +SELECT 'hello' +-- !query analysis +Project [hello AS hello#x] ++- OneRowRelation + + +-- !query +CREATE TEMPORARY VIEW v(c1 COMMENT 'hello') AS SELECT 1 +-- !query analysis +CreateViewCommand `v`, [(c1,Some(hello))], SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +DROP VIEW v +-- !query analysis +DropTempViewCommand v + + +-- !query +SELECT INTERVAL '1' YEAR +-- !query analysis +Project [INTERVAL '1' YEAR AS INTERVAL '1' YEAR#x] ++- OneRowRelation + + +-- !query +CREATE SCHEMA "myschema" +-- !query analysis +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 +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 +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"a2\"'", + "hint" : "" + } +} + + +-- !query +DROP TABLE "myview" +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myview\"'", + "hint" : "" + } +} + + +-- !query +DROP SCHEMA "myschema" +-- !query analysis +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/except-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/except-all.sql.out new file mode 100644 index 0000000000000..dedc3f93ba636 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/except-all.sql.out @@ -0,0 +1,670 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1) +-- !query analysis +CreateViewCommand `tab1`, SELECT * FROM VALUES + (0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1), false, false, LocalTempView, true + +- Project [c1#x] + +- SubqueryAlias tab1 + +- LocalRelation [c1#x] + + +-- !query +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1), (2), (2), (3), (5), (5), (null) AS tab2(c1) +-- !query analysis +CreateViewCommand `tab2`, SELECT * FROM VALUES + (1), (2), (2), (3), (5), (5), (null) AS tab2(c1), false, false, LocalTempView, true + +- Project [c1#x] + +- SubqueryAlias tab2 + +- LocalRelation [c1#x] + + +-- !query +CREATE TEMPORARY VIEW tab3 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (2, 3), + (2, 2) + AS tab3(k, v) +-- !query analysis +CreateViewCommand `tab3`, SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (2, 3), + (2, 2) + AS tab3(k, v), false, false, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias tab3 + +- LocalRelation [k#x, v#x] + + +-- !query +CREATE TEMPORARY VIEW tab4 AS SELECT * FROM VALUES + (1, 2), + (2, 3), + (2, 2), + (2, 2), + (2, 20) + AS tab4(k, v) +-- !query analysis +CreateViewCommand `tab4`, SELECT * FROM VALUES + (1, 2), + (2, 3), + (2, 2), + (2, 2), + (2, 20) + AS tab4(k, v), false, false, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM tab1 +EXCEPT ALL +SELECT * FROM tab2 +-- !query analysis +Except All true +:- Project [c1#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [c1#x]) +: +- Project [cast(c1#x as int) AS c1#x] +: +- Project [c1#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [c1#x] ++- Project [c1#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [c1#x]) + +- Project [cast(c1#x as int) AS c1#x] + +- Project [c1#x] + +- SubqueryAlias tab2 + +- LocalRelation [c1#x] + + +-- !query +SELECT * FROM tab1 +MINUS ALL +SELECT * FROM tab2 +-- !query analysis +Except All true +:- Project [c1#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [c1#x]) +: +- Project [cast(c1#x as int) AS c1#x] +: +- Project [c1#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [c1#x] ++- Project [c1#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [c1#x]) + +- Project [cast(c1#x as int) AS c1#x] + +- Project [c1#x] + +- SubqueryAlias tab2 + +- LocalRelation [c1#x] + + +-- !query +SELECT * FROM tab1 +EXCEPT ALL +SELECT * FROM tab2 WHERE c1 IS NOT NULL +-- !query analysis +Except All true +:- Project [c1#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [c1#x]) +: +- Project [cast(c1#x as int) AS c1#x] +: +- Project [c1#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [c1#x] ++- Project [c1#x] + +- Filter isnotnull(c1#x) + +- SubqueryAlias tab2 + +- View (`tab2`, [c1#x]) + +- Project [cast(c1#x as int) AS c1#x] + +- Project [c1#x] + +- SubqueryAlias tab2 + +- LocalRelation [c1#x] + + +-- !query +SELECT * FROM tab1 WHERE c1 > 5 +EXCEPT ALL +SELECT * FROM tab2 +-- !query analysis +Except All true +:- Project [c1#x] +: +- Filter (c1#x > 5) +: +- SubqueryAlias tab1 +: +- View (`tab1`, [c1#x]) +: +- Project [cast(c1#x as int) AS c1#x] +: +- Project [c1#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [c1#x] ++- Project [c1#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [c1#x]) + +- Project [cast(c1#x as int) AS c1#x] + +- Project [c1#x] + +- SubqueryAlias tab2 + +- LocalRelation [c1#x] + + +-- !query +SELECT * FROM tab1 +EXCEPT ALL +SELECT * FROM tab2 WHERE c1 > 6 +-- !query analysis +Except All true +:- Project [c1#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [c1#x]) +: +- Project [cast(c1#x as int) AS c1#x] +: +- Project [c1#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [c1#x] ++- Project [c1#x] + +- Filter (c1#x > 6) + +- SubqueryAlias tab2 + +- View (`tab2`, [c1#x]) + +- Project [cast(c1#x as int) AS c1#x] + +- Project [c1#x] + +- SubqueryAlias tab2 + +- LocalRelation [c1#x] + + +-- !query +SELECT * FROM tab1 +EXCEPT ALL +SELECT CAST(1 AS BIGINT) +-- !query analysis +Except All true +:- Project [cast(c1#x as bigint) AS c1#xL] +: +- Project [c1#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [c1#x]) +: +- Project [cast(c1#x as int) AS c1#x] +: +- Project [c1#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [c1#x] ++- Project [cast(1 as bigint) AS CAST(1 AS BIGINT)#xL] + +- OneRowRelation + + +-- !query +SELECT * FROM tab1 +EXCEPT ALL +SELECT array(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"ARRAY\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "EXCEPT ALL", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 45, + "fragment" : "SELECT * FROM tab1\nEXCEPT ALL\nSELECT array(1)" + } ] +} + + +-- !query +SELECT * FROM tab3 +EXCEPT ALL +SELECT * FROM tab4 +-- !query analysis +Except All true +:- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- View (`tab3`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM tab4 +EXCEPT ALL +SELECT * FROM tab3 +-- !query analysis +Except All true +:- Project [k#x, v#x] +: +- SubqueryAlias tab4 +: +- View (`tab4`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab4 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias tab3 + +- View (`tab3`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab3 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM tab4 +EXCEPT ALL +SELECT * FROM tab3 +INTERSECT DISTINCT +SELECT * FROM tab4 +-- !query analysis +Except All true +:- Project [k#x, v#x] +: +- SubqueryAlias tab4 +: +- View (`tab4`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab4 +: +- LocalRelation [k#x, v#x] ++- Intersect false + :- Project [k#x, v#x] + : +- SubqueryAlias tab3 + : +- View (`tab3`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias tab3 + : +- LocalRelation [k#x, v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM tab4 +EXCEPT ALL +SELECT * FROM tab3 +EXCEPT DISTINCT +SELECT * FROM tab4 +-- !query analysis +Except false +:- Except All true +: :- Project [k#x, v#x] +: : +- SubqueryAlias tab4 +: : +- View (`tab4`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab4 +: : +- LocalRelation [k#x, v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- View (`tab3`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM tab3 +EXCEPT ALL +SELECT * FROM tab4 +UNION ALL +SELECT * FROM tab3 +EXCEPT DISTINCT +SELECT * FROM tab4 +-- !query analysis +Except false +:- Union false, false +: :- Except All true +: : :- Project [k#x, v#x] +: : : +- SubqueryAlias tab3 +: : : +- View (`tab3`, [k#x,v#x]) +: : : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : : +- Project [k#x, v#x] +: : : +- SubqueryAlias tab3 +: : : +- LocalRelation [k#x, v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab4 +: : +- View (`tab4`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab4 +: : +- LocalRelation [k#x, v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- View (`tab3`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k FROM tab3 +EXCEPT ALL +SELECT k, v FROM tab4 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_COLUMNS_MISMATCH", + "sqlState" : "42826", + "messageParameters" : { + "firstNumColumns" : "1", + "invalidNumColumns" : "2", + "invalidOrdinalNum" : "second", + "operator" : "EXCEPT ALL" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 51, + "fragment" : "SELECT k FROM tab3\nEXCEPT ALL\nSELECT k, v FROM tab4" + } ] +} + + +-- !query +SELECT * FROM tab3 +EXCEPT ALL +SELECT * FROM tab4 +UNION +SELECT * FROM tab3 +EXCEPT DISTINCT +SELECT * FROM tab4 +-- !query analysis +Except false +:- Distinct +: +- Union false, false +: :- Except All true +: : :- Project [k#x, v#x] +: : : +- SubqueryAlias tab3 +: : : +- View (`tab3`, [k#x,v#x]) +: : : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : : +- Project [k#x, v#x] +: : : +- SubqueryAlias tab3 +: : : +- LocalRelation [k#x, v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab4 +: : +- View (`tab4`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab4 +: : +- LocalRelation [k#x, v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- View (`tab3`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM tab3 +MINUS ALL +SELECT * FROM tab4 +UNION +SELECT * FROM tab3 +MINUS DISTINCT +SELECT * FROM tab4 +-- !query analysis +Except false +:- Distinct +: +- Union false, false +: :- Except All true +: : :- Project [k#x, v#x] +: : : +- SubqueryAlias tab3 +: : : +- View (`tab3`, [k#x,v#x]) +: : : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : : +- Project [k#x, v#x] +: : : +- SubqueryAlias tab3 +: : : +- LocalRelation [k#x, v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab4 +: : +- View (`tab4`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab4 +: : +- LocalRelation [k#x, v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- View (`tab3`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM tab3 +EXCEPT ALL +SELECT * FROM tab4 +EXCEPT DISTINCT +SELECT * FROM tab3 +EXCEPT DISTINCT +SELECT * FROM tab4 +-- !query analysis +Except false +:- Except false +: :- Except All true +: : :- Project [k#x, v#x] +: : : +- SubqueryAlias tab3 +: : : +- View (`tab3`, [k#x,v#x]) +: : : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : : +- Project [k#x, v#x] +: : : +- SubqueryAlias tab3 +: : : +- LocalRelation [k#x, v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab4 +: : +- View (`tab4`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab4 +: : +- LocalRelation [k#x, v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- View (`tab3`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * +FROM (SELECT tab3.k, + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = tab4.k) +EXCEPT ALL +SELECT * +FROM (SELECT tab3.k, + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = tab4.k) +-- !query analysis +Except All true +:- Project [k#x, v#x] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [k#x, v#x] +: +- Join Inner, (k#x = k#x) +: :- SubqueryAlias tab3 +: : +- View (`tab3`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab3 +: : +- LocalRelation [k#x, v#x] +: +- SubqueryAlias tab4 +: +- View (`tab4`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab4 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [k#x, v#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias tab3 + : +- View (`tab3`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias tab3 + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * +FROM (SELECT tab3.k, + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = tab4.k) +EXCEPT ALL +SELECT * +FROM (SELECT tab4.v AS k, + tab3.k AS v + FROM tab3 + JOIN tab4 + ON tab3.k = tab4.k) +-- !query analysis +Except All true +:- Project [k#x, v#x] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [k#x, v#x] +: +- Join Inner, (k#x = k#x) +: :- SubqueryAlias tab3 +: : +- View (`tab3`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab3 +: : +- LocalRelation [k#x, v#x] +: +- SubqueryAlias tab4 +: +- View (`tab4`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab4 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [v#x AS k#x, k#x AS v#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias tab3 + : +- View (`tab3`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias tab3 + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT v FROM tab3 GROUP BY v +EXCEPT ALL +SELECT k FROM tab4 GROUP BY k +-- !query analysis +Except All true +:- Aggregate [v#x], [v#x] +: +- SubqueryAlias tab3 +: +- View (`tab3`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- LocalRelation [k#x, v#x] ++- Aggregate [k#x], [k#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +DROP VIEW IF EXISTS tab1 +-- !query analysis +DropTempViewCommand tab1 + + +-- !query +DROP VIEW IF EXISTS tab2 +-- !query analysis +DropTempViewCommand tab2 + + +-- !query +DROP VIEW IF EXISTS tab3 +-- !query analysis +DropTempViewCommand tab3 + + +-- !query +DROP VIEW IF EXISTS tab4 +-- !query analysis +DropTempViewCommand tab4 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/except.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/except.sql.out new file mode 100644 index 0000000000000..3c45f47174394 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/except.sql.out @@ -0,0 +1,323 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", NULL) + as t1(k, v) +-- !query analysis +CreateViewCommand `t1`, select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", NULL) + as t1(k, v), false, false, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias t1 + +- LocalRelation [k#x, v#x] + + +-- !query +create temporary view t2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5), + ("one", NULL), + (NULL, 5) + as t2(k, v) +-- !query analysis +CreateViewCommand `t2`, select * from values + ("one", 1), + ("two", 22), + ("one", 5), + ("one", NULL), + (NULL, 5) + as t2(k, v), false, false, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias t2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM t1 EXCEPT SELECT * FROM t2 +-- !query analysis +Except false +:- Project [k#x, v#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [k#x,v#x]) +: +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias t1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias t2 + +- View (`t2`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias t2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM t1 EXCEPT SELECT * FROM t1 where v <> 1 and v <> 2 +-- !query analysis +Except false +:- Project [k#x, v#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [k#x,v#x]) +: +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias t1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- Filter (NOT (v#x = 1) AND NOT (v#x = 2)) + +- SubqueryAlias t1 + +- View (`t1`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias t1 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM t1 where v <> 1 and v <> 22 EXCEPT SELECT * FROM t1 where v <> 2 and v >= 3 +-- !query analysis +Except false +:- Project [k#x, v#x] +: +- Filter (NOT (v#x = 1) AND NOT (v#x = 22)) +: +- SubqueryAlias t1 +: +- View (`t1`, [k#x,v#x]) +: +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias t1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- Filter (NOT (v#x = 2) AND (v#x >= 3)) + +- SubqueryAlias t1 + +- View (`t1`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias t1 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT t1.* FROM t1, t2 where t1.k = t2.k +EXCEPT +SELECT t1.* FROM t1, t2 where t1.k = t2.k and t1.k != 'one' +-- !query analysis +Except false +:- Project [k#x, v#x] +: +- Filter (k#x = k#x) +: +- Join Inner +: :- SubqueryAlias t1 +: : +- View (`t1`, [k#x,v#x]) +: : +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias t1 +: : +- LocalRelation [k#x, v#x] +: +- SubqueryAlias t2 +: +- View (`t2`, [k#x,v#x]) +: +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias t2 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- Filter ((k#x = k#x) AND NOT (k#x = one)) + +- Join Inner + :- SubqueryAlias t1 + : +- View (`t1`, [k#x,v#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias t1 + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias t2 + +- View (`t2`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias t2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM t2 where v >= 1 and v <> 22 EXCEPT SELECT * FROM t1 +-- !query analysis +Except false +:- Project [k#x, v#x] +: +- Filter ((v#x >= 1) AND NOT (v#x = 22)) +: +- SubqueryAlias t2 +: +- View (`t2`, [k#x,v#x]) +: +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias t2 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias t1 + +- View (`t1`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias t1 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT (SELECT min(k) FROM t2 WHERE t2.k = t1.k) min_t2 FROM t1 +MINUS +SELECT (SELECT min(k) FROM t2) abs_min_t2 FROM t1 WHERE t1.k = 'one' +-- !query analysis +Except false +:- Project [scalar-subquery#x [k#x] AS min_t2#x] +: : +- Aggregate [min(k#x) AS min(k)#x] +: : +- Filter (k#x = outer(k#x)) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [k#x,v#x]) +: : +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [k#x, v#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [k#x,v#x]) +: +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias t1 +: +- LocalRelation [k#x, v#x] ++- Project [scalar-subquery#x [] AS abs_min_t2#x] + : +- Aggregate [min(k#x) AS min(k)#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [k#x,v#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias t2 + : +- LocalRelation [k#x, v#x] + +- Filter (k#x = one) + +- SubqueryAlias t1 + +- View (`t1`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias t1 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT t1.k +FROM t1 +WHERE t1.v <= (SELECT max(t2.v) + FROM t2 + WHERE t2.k = t1.k) +MINUS +SELECT t1.k +FROM t1 +WHERE t1.v >= (SELECT min(t2.v) + FROM t2 + WHERE t2.k = t1.k) +-- !query analysis +Except false +:- Project [k#x] +: +- Filter (v#x <= scalar-subquery#x [k#x]) +: : +- Aggregate [max(v#x) AS max(v)#x] +: : +- Filter (k#x = outer(k#x)) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [k#x,v#x]) +: : +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [k#x, v#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [k#x,v#x]) +: +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias t1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x] + +- Filter (v#x >= scalar-subquery#x [k#x]) + : +- Aggregate [min(v#x) AS min(v)#x] + : +- Filter (k#x = outer(k#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [k#x,v#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias t2 + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias t1 + +- View (`t1`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias t1 + +- LocalRelation [k#x, v#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t3 AS VALUES (decimal(1)) tbl(v) +-- !query analysis +CreateViewCommand `t3`, VALUES (decimal(1)) tbl(v), false, true, LocalTempView, true + +- SubqueryAlias tbl + +- LocalRelation [v#x] + + +-- !query +SELECT t.v FROM ( + SELECT v FROM t3 + EXCEPT + SELECT v + v AS v FROM t3 +) t +-- !query analysis +Project [v#x] ++- SubqueryAlias t + +- Except false + :- Project [cast(v#x as decimal(11,0)) AS v#x] + : +- Project [v#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [v#x]) + : +- Project [cast(v#x as decimal(10,0)) AS v#x] + : +- SubqueryAlias tbl + : +- LocalRelation [v#x] + +- Project [(v#x + v#x) AS v#x] + +- SubqueryAlias t3 + +- View (`t3`, [v#x]) + +- Project [cast(v#x as decimal(10,0)) AS v#x] + +- SubqueryAlias tbl + +- LocalRelation [v#x] + + +-- !query +SELECT SUM(t.v) FROM ( + SELECT v FROM t3 + EXCEPT + SELECT v + v AS v FROM t3 +) t +-- !query analysis +Aggregate [sum(v#x) AS sum(v)#x] ++- SubqueryAlias t + +- Except false + :- Project [cast(v#x as decimal(11,0)) AS v#x] + : +- Project [v#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [v#x]) + : +- Project [cast(v#x as decimal(10,0)) AS v#x] + : +- SubqueryAlias tbl + : +- LocalRelation [v#x] + +- Project [(v#x + v#x) AS v#x] + +- SubqueryAlias t3 + +- View (`t3`, [v#x]) + +- Project [cast(v#x as decimal(10,0)) AS v#x] + +- SubqueryAlias tbl + +- LocalRelation [v#x] + + +-- !query +DROP VIEW IF EXISTS t1 +-- !query analysis +DropTempViewCommand t1 + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query analysis +DropTempViewCommand t2 + + +-- !query +DROP VIEW IF EXISTS t3 +-- !query analysis +DropTempViewCommand t3 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/explain-aqe.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/explain-aqe.sql.out new file mode 100644 index 0000000000000..8189f1fc7d1c5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/explain-aqe.sql.out @@ -0,0 +1,247 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE table explain_temp1 (key int, val int) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`explain_temp1`, false + + +-- !query +CREATE table explain_temp2 (key int, val int) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`explain_temp2`, false + + +-- !query +CREATE table explain_temp3 (key int, val int) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`explain_temp3`, false + + +-- !query +CREATE table explain_temp4 (key int, val string) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`explain_temp4`, false + + +-- !query +CREATE table explain_temp5 (key int) USING PARQUET PARTITIONED BY(val string) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`explain_temp5`, false + + +-- !query +SET spark.sql.codegen.wholeStage = true +-- !query analysis +SetCommand (spark.sql.codegen.wholeStage,Some(true)) + + +-- !query +EXPLAIN EXTENDED + SELECT sum(distinct val) + FROM explain_temp1 +-- !query analysis +ExplainCommand 'Project [unresolvedalias('sum(distinct 'val), None)], ExtendedMode + + +-- !query +EXPLAIN FORMATTED + SELECT key, max(val) + FROM explain_temp1 + WHERE key > 0 + GROUP BY key + ORDER BY key +-- !query analysis +ExplainCommand 'Sort ['key ASC NULLS FIRST], true, FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT key, max(val) + FROM explain_temp1 + WHERE key > 0 + GROUP BY key + HAVING max(val) > 0 +-- !query analysis +ExplainCommand 'UnresolvedHaving ('max('val) > 0), FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT key, val FROM explain_temp1 WHERE key > 0 + UNION + SELECT key, val FROM explain_temp1 WHERE key > 1 +-- !query analysis +ExplainCommand 'Distinct, FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 a, + explain_temp2 b + WHERE a.key = b.key +-- !query analysis +ExplainCommand 'Project [*], FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 a + LEFT OUTER JOIN explain_temp2 b + ON a.key = b.key +-- !query analysis +ExplainCommand 'Project [*], FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 + WHERE key = (SELECT max(key) + FROM explain_temp2 + WHERE key = (SELECT max(key) + FROM explain_temp3 + WHERE val > 0) + AND val = 2) + AND val > 3 +-- !query analysis +ExplainCommand 'Project [*], FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 + WHERE key = (SELECT max(key) + FROM explain_temp2 + WHERE val > 0) + OR + key = (SELECT avg(key) + FROM explain_temp3 + WHERE val > 0) +-- !query analysis +ExplainCommand 'Project [*], FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT (SELECT Avg(key) FROM explain_temp1) + (SELECT Avg(key) FROM explain_temp1) + FROM explain_temp1 +-- !query analysis +ExplainCommand 'Project [unresolvedalias((scalar-subquery#x [] + scalar-subquery#x []), None)], FormattedMode + + +-- !query +EXPLAIN FORMATTED + WITH cte1 AS ( + SELECT * + FROM explain_temp1 + WHERE key > 10 + ) + SELECT * FROM cte1 a, cte1 b WHERE a.key = b.key +-- !query analysis +ExplainCommand CTE [cte1], FormattedMode + + +-- !query +EXPLAIN FORMATTED + WITH cte1 AS ( + SELECT key, max(val) + FROM explain_temp1 + WHERE key > 10 + GROUP BY key + ) + SELECT * FROM cte1 a, cte1 b WHERE a.key = b.key +-- !query analysis +ExplainCommand CTE [cte1], FormattedMode + + +-- !query +EXPLAIN FORMATTED + CREATE VIEW explain_view AS + SELECT key, val FROM explain_temp1 +-- !query analysis +ExplainCommand 'CreateView SELECT key, val FROM explain_temp1, false, false, FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT + COUNT(val) + SUM(key) as TOTAL, + COUNT(key) FILTER (WHERE val > 1) + FROM explain_temp1 +-- !query analysis +ExplainCommand 'Project [('COUNT('val) + 'SUM('key)) AS TOTAL#x, unresolvedalias('COUNT('key, ('val > 1)), None)], FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT key, sort_array(collect_set(val))[0] + FROM explain_temp4 + GROUP BY key +-- !query analysis +ExplainCommand 'Aggregate ['key], ['key, unresolvedalias('sort_array('collect_set('val))[0], None)], FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT key, MIN(val) + FROM explain_temp4 + GROUP BY key +-- !query analysis +ExplainCommand 'Aggregate ['key], ['key, unresolvedalias('MIN('val), None)], FormattedMode + + +-- !query +EXPLAIN EXTENDED INSERT INTO TABLE explain_temp5 SELECT * FROM explain_temp4 +-- !query analysis +ExplainCommand 'InsertIntoStatement 'UnresolvedRelation [explain_temp5], [], false, false, false, ExtendedMode + + +-- !query +DROP TABLE explain_temp1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`explain_temp1`, false, false, false + + +-- !query +DROP TABLE explain_temp2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`explain_temp2`, false, false, false + + +-- !query +DROP TABLE explain_temp3 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`explain_temp3`, false, false, false + + +-- !query +DROP TABLE explain_temp4 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`explain_temp4`, false, false, false + + +-- !query +DROP TABLE explain_temp5 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`explain_temp5`, false, false, false + + +-- !query +CREATE table t(v array) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +EXPLAIN SELECT * FROM t WHERE v IN (array('a'), null) +-- !query analysis +ExplainCommand 'Project [*], SimpleMode + + +-- !query +DROP TABLE t +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/explain-cbo.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/explain-cbo.sql.out new file mode 100644 index 0000000000000..88b798b72e2c6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/explain-cbo.sql.out @@ -0,0 +1,55 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE explain_temp1(a INT, b INT) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`explain_temp1`, false + + +-- !query +CREATE TABLE explain_temp2(c INT, d INT) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`explain_temp2`, false + + +-- !query +ANALYZE TABLE explain_temp1 COMPUTE STATISTICS FOR ALL COLUMNS +-- !query analysis +AnalyzeColumnCommand `spark_catalog`.`default`.`explain_temp1`, true + + +-- !query +ANALYZE TABLE explain_temp2 COMPUTE STATISTICS FOR ALL COLUMNS +-- !query analysis +AnalyzeColumnCommand `spark_catalog`.`default`.`explain_temp2`, true + + +-- !query +EXPLAIN COST WITH max_store_sales AS +( + SELECT max(csales) tpcds_cmax + FROM ( + SELECT sum(b) csales + FROM explain_temp1 WHERE a < 100 + ) x +), +best_ss_customer AS +( + SELECT c + FROM explain_temp2 + WHERE d > (SELECT * FROM max_store_sales) +) +SELECT c FROM best_ss_customer +-- !query analysis +ExplainCommand CTE [max_store_sales, best_ss_customer], CostMode + + +-- !query +DROP TABLE explain_temp1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`explain_temp1`, false, false, false + + +-- !query +DROP TABLE explain_temp2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`explain_temp2`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/explain.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/explain.sql.out new file mode 100644 index 0000000000000..8189f1fc7d1c5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/explain.sql.out @@ -0,0 +1,247 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE table explain_temp1 (key int, val int) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`explain_temp1`, false + + +-- !query +CREATE table explain_temp2 (key int, val int) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`explain_temp2`, false + + +-- !query +CREATE table explain_temp3 (key int, val int) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`explain_temp3`, false + + +-- !query +CREATE table explain_temp4 (key int, val string) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`explain_temp4`, false + + +-- !query +CREATE table explain_temp5 (key int) USING PARQUET PARTITIONED BY(val string) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`explain_temp5`, false + + +-- !query +SET spark.sql.codegen.wholeStage = true +-- !query analysis +SetCommand (spark.sql.codegen.wholeStage,Some(true)) + + +-- !query +EXPLAIN EXTENDED + SELECT sum(distinct val) + FROM explain_temp1 +-- !query analysis +ExplainCommand 'Project [unresolvedalias('sum(distinct 'val), None)], ExtendedMode + + +-- !query +EXPLAIN FORMATTED + SELECT key, max(val) + FROM explain_temp1 + WHERE key > 0 + GROUP BY key + ORDER BY key +-- !query analysis +ExplainCommand 'Sort ['key ASC NULLS FIRST], true, FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT key, max(val) + FROM explain_temp1 + WHERE key > 0 + GROUP BY key + HAVING max(val) > 0 +-- !query analysis +ExplainCommand 'UnresolvedHaving ('max('val) > 0), FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT key, val FROM explain_temp1 WHERE key > 0 + UNION + SELECT key, val FROM explain_temp1 WHERE key > 1 +-- !query analysis +ExplainCommand 'Distinct, FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 a, + explain_temp2 b + WHERE a.key = b.key +-- !query analysis +ExplainCommand 'Project [*], FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 a + LEFT OUTER JOIN explain_temp2 b + ON a.key = b.key +-- !query analysis +ExplainCommand 'Project [*], FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 + WHERE key = (SELECT max(key) + FROM explain_temp2 + WHERE key = (SELECT max(key) + FROM explain_temp3 + WHERE val > 0) + AND val = 2) + AND val > 3 +-- !query analysis +ExplainCommand 'Project [*], FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 + WHERE key = (SELECT max(key) + FROM explain_temp2 + WHERE val > 0) + OR + key = (SELECT avg(key) + FROM explain_temp3 + WHERE val > 0) +-- !query analysis +ExplainCommand 'Project [*], FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT (SELECT Avg(key) FROM explain_temp1) + (SELECT Avg(key) FROM explain_temp1) + FROM explain_temp1 +-- !query analysis +ExplainCommand 'Project [unresolvedalias((scalar-subquery#x [] + scalar-subquery#x []), None)], FormattedMode + + +-- !query +EXPLAIN FORMATTED + WITH cte1 AS ( + SELECT * + FROM explain_temp1 + WHERE key > 10 + ) + SELECT * FROM cte1 a, cte1 b WHERE a.key = b.key +-- !query analysis +ExplainCommand CTE [cte1], FormattedMode + + +-- !query +EXPLAIN FORMATTED + WITH cte1 AS ( + SELECT key, max(val) + FROM explain_temp1 + WHERE key > 10 + GROUP BY key + ) + SELECT * FROM cte1 a, cte1 b WHERE a.key = b.key +-- !query analysis +ExplainCommand CTE [cte1], FormattedMode + + +-- !query +EXPLAIN FORMATTED + CREATE VIEW explain_view AS + SELECT key, val FROM explain_temp1 +-- !query analysis +ExplainCommand 'CreateView SELECT key, val FROM explain_temp1, false, false, FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT + COUNT(val) + SUM(key) as TOTAL, + COUNT(key) FILTER (WHERE val > 1) + FROM explain_temp1 +-- !query analysis +ExplainCommand 'Project [('COUNT('val) + 'SUM('key)) AS TOTAL#x, unresolvedalias('COUNT('key, ('val > 1)), None)], FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT key, sort_array(collect_set(val))[0] + FROM explain_temp4 + GROUP BY key +-- !query analysis +ExplainCommand 'Aggregate ['key], ['key, unresolvedalias('sort_array('collect_set('val))[0], None)], FormattedMode + + +-- !query +EXPLAIN FORMATTED + SELECT key, MIN(val) + FROM explain_temp4 + GROUP BY key +-- !query analysis +ExplainCommand 'Aggregate ['key], ['key, unresolvedalias('MIN('val), None)], FormattedMode + + +-- !query +EXPLAIN EXTENDED INSERT INTO TABLE explain_temp5 SELECT * FROM explain_temp4 +-- !query analysis +ExplainCommand 'InsertIntoStatement 'UnresolvedRelation [explain_temp5], [], false, false, false, ExtendedMode + + +-- !query +DROP TABLE explain_temp1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`explain_temp1`, false, false, false + + +-- !query +DROP TABLE explain_temp2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`explain_temp2`, false, false, false + + +-- !query +DROP TABLE explain_temp3 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`explain_temp3`, false, false, false + + +-- !query +DROP TABLE explain_temp4 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`explain_temp4`, false, false, false + + +-- !query +DROP TABLE explain_temp5 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`explain_temp5`, false, false, false + + +-- !query +CREATE table t(v array) USING PARQUET +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +EXPLAIN SELECT * FROM t WHERE v IN (array('a'), null) +-- !query analysis +ExplainCommand 'Project [*], SimpleMode + + +-- !query +DROP TABLE t +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t`, false, false, false 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 new file mode 100644 index 0000000000000..6085457deaa06 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/extract.sql.out @@ -0,0 +1,1441 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +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, 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] + +- OneRowRelation + + +-- !query +select extract(year from c), extract(year from ntz), extract(year from i) from t +-- !query analysis +Project [extract(year, c#x) AS extract(year FROM c)#x, extract(year, ntz#x) AS extract(year FROM ntz)#x, extract(year, i#x) AS extract(year FROM 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] + +- OneRowRelation + + +-- !query +select extract(y from c), extract(y from ntz), extract(y from i) from t +-- !query analysis +Project [extract(y, c#x) AS extract(y FROM c)#x, extract(y, ntz#x) AS extract(y FROM ntz)#x, extract(y, i#x) AS extract(y FROM 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] + +- OneRowRelation + + +-- !query +select extract(years from c), extract(years from ntz), extract(years from i) from t +-- !query analysis +Project [extract(years, c#x) AS extract(years FROM c)#x, extract(years, ntz#x) AS extract(years FROM ntz)#x, extract(years, i#x) AS extract(years FROM 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] + +- OneRowRelation + + +-- !query +select extract(yr from c), extract(yr from ntz), extract(yr from i) from t +-- !query analysis +Project [extract(yr, c#x) AS extract(yr FROM c)#x, extract(yr, ntz#x) AS extract(yr FROM ntz)#x, extract(yr, i#x) AS extract(yr FROM 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] + +- OneRowRelation + + +-- !query +select extract(yrs from c), extract(yrs from ntz), extract(yrs from i) from t +-- !query analysis +Project [extract(yrs, c#x) AS extract(yrs FROM c)#x, extract(yrs, ntz#x) AS extract(yrs FROM ntz)#x, extract(yrs, i#x) AS extract(yrs FROM 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] + +- OneRowRelation + + +-- !query +select extract(yearofweek from c), extract(yearofweek from ntz) from t +-- !query analysis +Project [extract(yearofweek, c#x) AS extract(yearofweek FROM c)#x, extract(yearofweek, ntz#x) AS extract(yearofweek FROM 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] + +- OneRowRelation + + +-- !query +select extract(quarter from c), extract(quarter from ntz) from t +-- !query analysis +Project [extract(quarter, c#x) AS extract(quarter FROM c)#x, extract(quarter, ntz#x) AS extract(quarter FROM 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] + +- OneRowRelation + + +-- !query +select extract(qtr from c), extract(qtr from ntz) from t +-- !query analysis +Project [extract(qtr, c#x) AS extract(qtr FROM c)#x, extract(qtr, ntz#x) AS extract(qtr FROM 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] + +- OneRowRelation + + +-- !query +select extract(month from c), extract(month from ntz), extract(month from i) from t +-- !query analysis +Project [extract(month, c#x) AS extract(month FROM c)#x, extract(month, ntz#x) AS extract(month FROM ntz)#x, extract(month, i#x) AS extract(month FROM 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] + +- OneRowRelation + + +-- !query +select extract(mon from c), extract(mon from ntz), extract(mon from i) from t +-- !query analysis +Project [extract(mon, c#x) AS extract(mon FROM c)#x, extract(mon, ntz#x) AS extract(mon FROM ntz)#x, extract(mon, i#x) AS extract(mon FROM 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] + +- OneRowRelation + + +-- !query +select extract(mons from c), extract(mons from ntz), extract(mons from i) from t +-- !query analysis +Project [extract(mons, c#x) AS extract(mons FROM c)#x, extract(mons, ntz#x) AS extract(mons FROM ntz)#x, extract(mons, i#x) AS extract(mons FROM 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] + +- OneRowRelation + + +-- !query +select extract(months from c), extract(months from ntz), extract(months from i) from t +-- !query analysis +Project [extract(months, c#x) AS extract(months FROM c)#x, extract(months, ntz#x) AS extract(months FROM ntz)#x, extract(months, i#x) AS extract(months FROM 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] + +- OneRowRelation + + +-- !query +select extract(week from c), extract(week from ntz) from t +-- !query analysis +Project [extract(week, c#x) AS extract(week FROM c)#x, extract(week, ntz#x) AS extract(week FROM 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] + +- OneRowRelation + + +-- !query +select extract(w from c), extract(w from ntz) from t +-- !query analysis +Project [extract(w, c#x) AS extract(w FROM c)#x, extract(w, ntz#x) AS extract(w FROM 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] + +- OneRowRelation + + +-- !query +select extract(weeks from c), extract(weeks from ntz) from t +-- !query analysis +Project [extract(weeks, c#x) AS extract(weeks FROM c)#x, extract(weeks, ntz#x) AS extract(weeks FROM 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] + +- OneRowRelation + + +-- !query +select extract(day from c), extract(day from ntz), extract(day from j) from t +-- !query analysis +Project [extract(day, c#x) AS extract(day FROM c)#x, extract(day, ntz#x) AS extract(day FROM ntz)#x, extract(day, j#x) AS extract(day FROM 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] + +- OneRowRelation + + +-- !query +select extract(d from c), extract(d from ntz), extract(d from j) from t +-- !query analysis +Project [extract(d, c#x) AS extract(d FROM c)#x, extract(d, ntz#x) AS extract(d FROM ntz)#x, extract(d, j#x) AS extract(d FROM 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] + +- OneRowRelation + + +-- !query +select extract(days from c), extract(days from ntz), extract(days from j) from t +-- !query analysis +Project [extract(days, c#x) AS extract(days FROM c)#x, extract(days, ntz#x) AS extract(days FROM ntz)#x, extract(days, j#x) AS extract(days FROM 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] + +- OneRowRelation + + +-- !query +select extract(dayofweek from c), extract(dayofweek from ntz) from t +-- !query analysis +Project [extract(dayofweek, c#x) AS extract(dayofweek FROM c)#x, extract(dayofweek, ntz#x) AS extract(dayofweek FROM 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] + +- OneRowRelation + + +-- !query +select extract(dow from c), extract(dow from ntz) from t +-- !query analysis +Project [extract(dow, c#x) AS extract(dow FROM c)#x, extract(dow, ntz#x) AS extract(dow FROM 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] + +- OneRowRelation + + +-- !query +select extract(dayofweek_iso from c), extract(dayofweek_iso from ntz) from t +-- !query analysis +Project [extract(dayofweek_iso, c#x) AS extract(dayofweek_iso FROM c)#x, extract(dayofweek_iso, ntz#x) AS extract(dayofweek_iso FROM 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] + +- OneRowRelation + + +-- !query +select extract(dow_iso from c), extract(dow_iso from ntz) from t +-- !query analysis +Project [extract(dow_iso, c#x) AS extract(dow_iso FROM c)#x, extract(dow_iso, ntz#x) AS extract(dow_iso FROM 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] + +- OneRowRelation + + +-- !query +select extract(doy from c), extract(doy from ntz) from t +-- !query analysis +Project [extract(doy, c#x) AS extract(doy FROM c)#x, extract(doy, ntz#x) AS extract(doy FROM 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] + +- OneRowRelation + + +-- !query +select extract(hour from c), extract(hour from ntz), extract(hour from j) from t +-- !query analysis +Project [extract(hour, c#x) AS extract(hour FROM c)#x, extract(hour, ntz#x) AS extract(hour FROM ntz)#x, extract(hour, j#x) AS extract(hour FROM 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] + +- OneRowRelation + + +-- !query +select extract(h from c), extract(h from ntz), extract(h from j) from t +-- !query analysis +Project [extract(h, c#x) AS extract(h FROM c)#x, extract(h, ntz#x) AS extract(h FROM ntz)#x, extract(h, j#x) AS extract(h FROM 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] + +- OneRowRelation + + +-- !query +select extract(hours from c), extract(hours from ntz), extract(hours from j) from t +-- !query analysis +Project [extract(hours, c#x) AS extract(hours FROM c)#x, extract(hours, ntz#x) AS extract(hours FROM ntz)#x, extract(hours, j#x) AS extract(hours FROM 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] + +- OneRowRelation + + +-- !query +select extract(hr from c), extract(hr from ntz), extract(hr from j) from t +-- !query analysis +Project [extract(hr, c#x) AS extract(hr FROM c)#x, extract(hr, ntz#x) AS extract(hr FROM ntz)#x, extract(hr, j#x) AS extract(hr FROM 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] + +- OneRowRelation + + +-- !query +select extract(hrs from c), extract(hrs from ntz), extract(hrs from j) from t +-- !query analysis +Project [extract(hrs, c#x) AS extract(hrs FROM c)#x, extract(hrs, ntz#x) AS extract(hrs FROM ntz)#x, extract(hrs, j#x) AS extract(hrs FROM 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] + +- OneRowRelation + + +-- !query +select extract(minute from c), extract(minute from ntz), extract(minute from j) from t +-- !query analysis +Project [extract(minute, c#x) AS extract(minute FROM c)#x, extract(minute, ntz#x) AS extract(minute FROM ntz)#x, extract(minute, j#x) AS extract(minute FROM 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] + +- OneRowRelation + + +-- !query +select extract(m from c), extract(m from ntz), extract(m from j) from t +-- !query analysis +Project [extract(m, c#x) AS extract(m FROM c)#x, extract(m, ntz#x) AS extract(m FROM ntz)#x, extract(m, j#x) AS extract(m FROM 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] + +- OneRowRelation + + +-- !query +select extract(min from c), extract(min from ntz), extract(min from j) from t +-- !query analysis +Project [extract(min, c#x) AS extract(min FROM c)#x, extract(min, ntz#x) AS extract(min FROM ntz)#x, extract(min, j#x) AS extract(min FROM 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] + +- OneRowRelation + + +-- !query +select extract(mins from c), extract(mins from ntz), extract(mins from j) from t +-- !query analysis +Project [extract(mins, c#x) AS extract(mins FROM c)#x, extract(mins, ntz#x) AS extract(mins FROM ntz)#x, extract(mins, j#x) AS extract(mins FROM 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] + +- OneRowRelation + + +-- !query +select extract(minutes from c), extract(minutes from ntz), extract(minutes from j) from t +-- !query analysis +Project [extract(minutes, c#x) AS extract(minutes FROM c)#x, extract(minutes, ntz#x) AS extract(minutes FROM ntz)#x, extract(minutes, j#x) AS extract(minutes FROM 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] + +- OneRowRelation + + +-- !query +select extract(second from c), extract(second from ntz), extract(second from j) from t +-- !query analysis +Project [extract(second, c#x) AS extract(second FROM c)#x, extract(second, ntz#x) AS extract(second FROM ntz)#x, extract(second, j#x) AS extract(second FROM 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] + +- OneRowRelation + + +-- !query +select extract(s from c), extract(s from ntz), extract(s from j) from t +-- !query analysis +Project [extract(s, c#x) AS extract(s FROM c)#x, extract(s, ntz#x) AS extract(s FROM ntz)#x, extract(s, j#x) AS extract(s FROM 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] + +- OneRowRelation + + +-- !query +select extract(sec from c), extract(sec from ntz), extract(sec from j) from t +-- !query analysis +Project [extract(sec, c#x) AS extract(sec FROM c)#x, extract(sec, ntz#x) AS extract(sec FROM ntz)#x, extract(sec, j#x) AS extract(sec FROM 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] + +- OneRowRelation + + +-- !query +select extract(seconds from c), extract(seconds from ntz), extract(seconds from j) from t +-- !query analysis +Project [extract(seconds, c#x) AS extract(seconds FROM c)#x, extract(seconds, ntz#x) AS extract(seconds FROM ntz)#x, extract(seconds, j#x) AS extract(seconds FROM 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] + +- OneRowRelation + + +-- !query +select extract(secs from c), extract(secs from ntz), extract(secs from j) from t +-- !query analysis +Project [extract(secs, c#x) AS extract(secs FROM c)#x, extract(secs, ntz#x) AS extract(secs FROM ntz)#x, extract(secs, j#x) AS extract(secs FROM 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] + +- OneRowRelation + + +-- !query +select extract(not_supported from c) from t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"c\"", + "field" : "`not_supported`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "extract(not_supported from c)" + } ] +} + + +-- !query +select extract(not_supported from i) from t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"i\"", + "field" : "`not_supported`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "extract(not_supported from i)" + } ] +} + + +-- !query +select extract(not_supported from j) from t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"j\"", + "field" : "`not_supported`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "extract(not_supported from j)" + } ] +} + + +-- !query +select date_part('year', c), date_part('year', ntz), date_part('year', i) from t +-- !query analysis +Project [date_part(year, c#x) AS date_part(year, c)#x, date_part(year, ntz#x) AS date_part(year, ntz)#x, date_part(year, i#x) AS date_part(year, 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] + +- OneRowRelation + + +-- !query +select date_part('y', c), date_part('y', ntz), date_part('y', i) from t +-- !query analysis +Project [date_part(y, c#x) AS date_part(y, c)#x, date_part(y, ntz#x) AS date_part(y, ntz)#x, date_part(y, i#x) AS date_part(y, 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] + +- OneRowRelation + + +-- !query +select date_part('years', c), date_part('years', ntz), date_part('years', i) from t +-- !query analysis +Project [date_part(years, c#x) AS date_part(years, c)#x, date_part(years, ntz#x) AS date_part(years, ntz)#x, date_part(years, i#x) AS date_part(years, 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] + +- OneRowRelation + + +-- !query +select date_part('yr', c), date_part('yr', ntz), date_part('yr', i) from t +-- !query analysis +Project [date_part(yr, c#x) AS date_part(yr, c)#x, date_part(yr, ntz#x) AS date_part(yr, ntz)#x, date_part(yr, i#x) AS date_part(yr, 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] + +- OneRowRelation + + +-- !query +select date_part('yrs', c), date_part('yrs', ntz), date_part('yrs', i) from t +-- !query analysis +Project [date_part(yrs, c#x) AS date_part(yrs, c)#x, date_part(yrs, ntz#x) AS date_part(yrs, ntz)#x, date_part(yrs, i#x) AS date_part(yrs, 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] + +- OneRowRelation + + +-- !query +select date_part('yearofweek', c), date_part('yearofweek', ntz) from t +-- !query analysis +Project [date_part(yearofweek, c#x) AS date_part(yearofweek, c)#x, date_part(yearofweek, ntz#x) AS date_part(yearofweek, 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] + +- OneRowRelation + + +-- !query +select date_part('quarter', c), date_part('quarter', ntz) from t +-- !query analysis +Project [date_part(quarter, c#x) AS date_part(quarter, c)#x, date_part(quarter, ntz#x) AS date_part(quarter, 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] + +- OneRowRelation + + +-- !query +select date_part('qtr', c), date_part('qtr', ntz) from t +-- !query analysis +Project [date_part(qtr, c#x) AS date_part(qtr, c)#x, date_part(qtr, ntz#x) AS date_part(qtr, 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] + +- OneRowRelation + + +-- !query +select date_part('month', c), date_part('month', ntz), date_part('month', i) from t +-- !query analysis +Project [date_part(month, c#x) AS date_part(month, c)#x, date_part(month, ntz#x) AS date_part(month, ntz)#x, date_part(month, i#x) AS date_part(month, 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] + +- OneRowRelation + + +-- !query +select date_part('mon', c), date_part('mon', ntz), date_part('mon', i) from t +-- !query analysis +Project [date_part(mon, c#x) AS date_part(mon, c)#x, date_part(mon, ntz#x) AS date_part(mon, ntz)#x, date_part(mon, i#x) AS date_part(mon, 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] + +- OneRowRelation + + +-- !query +select date_part('mons', c), date_part('mons', ntz), date_part('mons', i) from t +-- !query analysis +Project [date_part(mons, c#x) AS date_part(mons, c)#x, date_part(mons, ntz#x) AS date_part(mons, ntz)#x, date_part(mons, i#x) AS date_part(mons, 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] + +- OneRowRelation + + +-- !query +select date_part('months', c), date_part('months', ntz), date_part('months', i) from t +-- !query analysis +Project [date_part(months, c#x) AS date_part(months, c)#x, date_part(months, ntz#x) AS date_part(months, ntz)#x, date_part(months, i#x) AS date_part(months, 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] + +- OneRowRelation + + +-- !query +select date_part('week', c), date_part('week', ntz) from t +-- !query analysis +Project [date_part(week, c#x) AS date_part(week, c)#x, date_part(week, ntz#x) AS date_part(week, 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] + +- OneRowRelation + + +-- !query +select date_part('w', c), date_part('w', ntz) from t +-- !query analysis +Project [date_part(w, c#x) AS date_part(w, c)#x, date_part(w, ntz#x) AS date_part(w, 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] + +- OneRowRelation + + +-- !query +select date_part('weeks', c), date_part('weeks', ntz) from t +-- !query analysis +Project [date_part(weeks, c#x) AS date_part(weeks, c)#x, date_part(weeks, ntz#x) AS 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] + +- OneRowRelation + + +-- !query +select date_part('day', c), date_part('day', ntz), date_part('day', j) from t +-- !query analysis +Project [date_part(day, c#x) AS date_part(day, c)#x, date_part(day, ntz#x) AS date_part(day, ntz)#x, date_part(day, j#x) AS date_part(day, 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] + +- OneRowRelation + + +-- !query +select date_part('d', c), date_part('d', ntz), date_part('d', j) from t +-- !query analysis +Project [date_part(d, c#x) AS date_part(d, c)#x, date_part(d, ntz#x) AS date_part(d, ntz)#x, date_part(d, j#x) AS date_part(d, 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] + +- OneRowRelation + + +-- !query +select date_part('days', c), date_part('days', ntz), date_part('days', j) from t +-- !query analysis +Project [date_part(days, c#x) AS date_part(days, c)#x, date_part(days, ntz#x) AS date_part(days, ntz)#x, date_part(days, j#x) AS date_part(days, 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] + +- OneRowRelation + + +-- !query +select date_part('dayofweek', c), date_part('dayofweek', ntz) from t +-- !query analysis +Project [date_part(dayofweek, c#x) AS date_part(dayofweek, c)#x, date_part(dayofweek, ntz#x) AS date_part(dayofweek, 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] + +- OneRowRelation + + +-- !query +select date_part('dow', c), date_part('dow', ntz) from t +-- !query analysis +Project [date_part(dow, c#x) AS date_part(dow, c)#x, date_part(dow, ntz#x) AS date_part(dow, 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] + +- OneRowRelation + + +-- !query +select date_part('dayofweek_iso', c), date_part('dayofweek_iso', ntz) from t +-- !query analysis +Project [date_part(dayofweek_iso, c#x) AS date_part(dayofweek_iso, c)#x, date_part(dayofweek_iso, ntz#x) AS date_part(dayofweek_iso, 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] + +- OneRowRelation + + +-- !query +select date_part('dow_iso', c), date_part('dow_iso', ntz) from t +-- !query analysis +Project [date_part(dow_iso, c#x) AS date_part(dow_iso, c)#x, date_part(dow_iso, ntz#x) AS date_part(dow_iso, 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] + +- OneRowRelation + + +-- !query +select date_part('doy', c), date_part('doy', ntz) from t +-- !query analysis +Project [date_part(doy, c#x) AS date_part(doy, c)#x, date_part(doy, ntz#x) AS date_part(doy, 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] + +- OneRowRelation + + +-- !query +select date_part('hour', c), date_part('hour', ntz), date_part('hour', j) from t +-- !query analysis +Project [date_part(hour, c#x) AS date_part(hour, c)#x, date_part(hour, ntz#x) AS date_part(hour, ntz)#x, date_part(hour, j#x) AS date_part(hour, 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] + +- OneRowRelation + + +-- !query +select date_part('h', c), date_part('h', ntz), date_part('h', j) from t +-- !query analysis +Project [date_part(h, c#x) AS date_part(h, c)#x, date_part(h, ntz#x) AS date_part(h, ntz)#x, date_part(h, j#x) AS date_part(h, 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] + +- OneRowRelation + + +-- !query +select date_part('hours', c), date_part('hours', ntz), date_part('hours', j) from t +-- !query analysis +Project [date_part(hours, c#x) AS date_part(hours, c)#x, date_part(hours, ntz#x) AS date_part(hours, ntz)#x, date_part(hours, j#x) AS date_part(hours, 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] + +- OneRowRelation + + +-- !query +select date_part('hr', c), date_part('hr', ntz), date_part('hr', j) from t +-- !query analysis +Project [date_part(hr, c#x) AS date_part(hr, c)#x, date_part(hr, ntz#x) AS date_part(hr, ntz)#x, date_part(hr, j#x) AS date_part(hr, 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] + +- OneRowRelation + + +-- !query +select date_part('hrs', c), date_part('hrs', ntz), date_part('hrs', j) from t +-- !query analysis +Project [date_part(hrs, c#x) AS date_part(hrs, c)#x, date_part(hrs, ntz#x) AS date_part(hrs, ntz)#x, date_part(hrs, j#x) AS date_part(hrs, 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] + +- OneRowRelation + + +-- !query +select date_part('minute', c), date_part('minute', ntz), date_part('minute', j) from t +-- !query analysis +Project [date_part(minute, c#x) AS date_part(minute, c)#x, date_part(minute, ntz#x) AS date_part(minute, ntz)#x, date_part(minute, j#x) AS date_part(minute, 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] + +- OneRowRelation + + +-- !query +select date_part('m', c), date_part('m', ntz), date_part('m', j) from t +-- !query analysis +Project [date_part(m, c#x) AS date_part(m, c)#x, date_part(m, ntz#x) AS date_part(m, ntz)#x, date_part(m, j#x) AS date_part(m, 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] + +- OneRowRelation + + +-- !query +select date_part('min', c), date_part('min', ntz), date_part('min', j) from t +-- !query analysis +Project [date_part(min, c#x) AS date_part(min, c)#x, date_part(min, ntz#x) AS date_part(min, ntz)#x, date_part(min, j#x) AS date_part(min, 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] + +- OneRowRelation + + +-- !query +select date_part('mins', c), date_part('mins', ntz), date_part('mins', j) from t +-- !query analysis +Project [date_part(mins, c#x) AS date_part(mins, c)#x, date_part(mins, ntz#x) AS date_part(mins, ntz)#x, date_part(mins, j#x) AS date_part(mins, 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] + +- OneRowRelation + + +-- !query +select date_part('minutes', c), date_part('minutes', ntz), date_part('minutes', j) from t +-- !query analysis +Project [date_part(minutes, c#x) AS date_part(minutes, c)#x, date_part(minutes, ntz#x) AS date_part(minutes, ntz)#x, date_part(minutes, j#x) AS date_part(minutes, 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] + +- OneRowRelation + + +-- !query +select date_part('second', c), date_part('second', ntz), date_part('second', j) from t +-- !query analysis +Project [date_part(second, c#x) AS date_part(second, c)#x, date_part(second, ntz#x) AS date_part(second, ntz)#x, date_part(second, j#x) AS date_part(second, 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] + +- OneRowRelation + + +-- !query +select date_part('s', c), date_part('s', ntz), date_part('s', j) from t +-- !query analysis +Project [date_part(s, c#x) AS date_part(s, c)#x, date_part(s, ntz#x) AS date_part(s, ntz)#x, date_part(s, j#x) AS date_part(s, 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] + +- OneRowRelation + + +-- !query +select date_part('sec', c), date_part('sec', ntz), date_part('sec', j) from t +-- !query analysis +Project [date_part(sec, c#x) AS date_part(sec, c)#x, date_part(sec, ntz#x) AS date_part(sec, ntz)#x, date_part(sec, j#x) AS date_part(sec, 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] + +- OneRowRelation + + +-- !query +select date_part('seconds', c), date_part('seconds', ntz), date_part('seconds', j) from t +-- !query analysis +Project [date_part(seconds, c#x) AS date_part(seconds, c)#x, date_part(seconds, ntz#x) AS date_part(seconds, ntz)#x, date_part(seconds, j#x) AS date_part(seconds, 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] + +- OneRowRelation + + +-- !query +select date_part('secs', c), date_part('secs', ntz), date_part('secs', j) from t +-- !query analysis +Project [date_part(secs, c#x) AS date_part(secs, c)#x, date_part(secs, ntz#x) AS date_part(secs, ntz)#x, date_part(secs, j#x) AS date_part(secs, 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] + +- OneRowRelation + + +-- !query +select date_part('not_supported', c) from t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"c\"", + "field" : "`not_supported`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "date_part('not_supported', c)" + } ] +} + + +-- !query +select date_part(c, c) from t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1100", + "messageParameters" : { + "argName" : "field", + "funcName" : "date_part", + "requiredType" : "string" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "date_part(c, c)" + } ] +} + + +-- !query +select date_part(null, c) from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select date_part(i, i) from t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1100", + "messageParameters" : { + "argName" : "field", + "funcName" : "date_part", + "requiredType" : "string" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "date_part(i, i)" + } ] +} + + +-- !query +select date_part(null, i) from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select extract('year', c) from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select extract('quarter', c) from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select extract('month', c) from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select extract('week', c) from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select extract('day', c) from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select extract('days', c) from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select extract('dayofweek', c) from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select extract('dow', c) from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select extract('doy', c) from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select extract('hour', c) from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select extract('minute', c) from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select extract('second', c) from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select c - j from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select day(c - j) from t +-- !query analysis +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] + +- OneRowRelation + + +-- !query +select extract(day from c - j) from t +-- !query analysis +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] + +- 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] ++- 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] + +- 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] ++- 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] + +- OneRowRelation + + +-- !query +select extract(YEAR from interval '2-1' YEAR TO MONTH) +-- !query analysis +Project [extract(YEAR, INTERVAL '2-1' YEAR TO MONTH) AS extract(YEAR FROM INTERVAL '2-1' YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select date_part('YEAR', interval '2-1' YEAR TO MONTH) +-- !query analysis +Project [date_part(YEAR, INTERVAL '2-1' YEAR TO MONTH) AS date_part(YEAR, INTERVAL '2-1' YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select extract(YEAR from -interval '2-1' YEAR TO MONTH) +-- !query analysis +Project [extract(YEAR, -INTERVAL '2-1' YEAR TO MONTH) AS extract(YEAR FROM (- INTERVAL '2-1' YEAR TO MONTH))#x] ++- OneRowRelation + + +-- !query +select extract(MONTH from interval '2-1' YEAR TO MONTH) +-- !query analysis +Project [extract(MONTH, INTERVAL '2-1' YEAR TO MONTH) AS extract(MONTH FROM INTERVAL '2-1' YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select date_part('MONTH', interval '2-1' YEAR TO MONTH) +-- !query analysis +Project [date_part(MONTH, INTERVAL '2-1' YEAR TO MONTH) AS date_part(MONTH, INTERVAL '2-1' YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select extract(MONTH from -interval '2-1' YEAR TO MONTH) +-- !query analysis +Project [extract(MONTH, -INTERVAL '2-1' YEAR TO MONTH) AS extract(MONTH FROM (- INTERVAL '2-1' YEAR TO MONTH))#x] ++- OneRowRelation + + +-- !query +select date_part(NULL, interval '2-1' YEAR TO MONTH) +-- !query analysis +Project [date_part(null, INTERVAL '2-1' YEAR TO MONTH) AS date_part(NULL, INTERVAL '2-1' YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select extract(DAY from interval '2-1' YEAR TO MONTH) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"INTERVAL '2-1' YEAR TO MONTH\"", + "field" : "`DAY`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "extract(DAY from interval '2-1' YEAR TO MONTH)" + } ] +} + + +-- !query +select date_part('DAY', interval '2-1' YEAR TO MONTH) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"INTERVAL '2-1' YEAR TO MONTH\"", + "field" : "`DAY`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "date_part('DAY', interval '2-1' YEAR TO MONTH)" + } ] +} + + +-- !query +select date_part('not_supported', interval '2-1' YEAR TO MONTH) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"INTERVAL '2-1' YEAR TO MONTH\"", + "field" : "`not_supported`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "date_part('not_supported', interval '2-1' YEAR TO MONTH)" + } ] +} + + +-- !query +select extract(DAY from interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +Project [extract(DAY, INTERVAL '123 12:34:56.789123' DAY TO SECOND) AS extract(DAY FROM INTERVAL '123 12:34:56.789123' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select date_part('DAY', interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +Project [date_part(DAY, INTERVAL '123 12:34:56.789123' DAY TO SECOND) AS date_part(DAY, INTERVAL '123 12:34:56.789123' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select extract(DAY from -interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +Project [extract(DAY, -INTERVAL '123 12:34:56.789123' DAY TO SECOND) AS extract(DAY FROM (- INTERVAL '123 12:34:56.789123' DAY TO SECOND))#x] ++- OneRowRelation + + +-- !query +select extract(HOUR from interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +Project [extract(HOUR, INTERVAL '123 12:34:56.789123' DAY TO SECOND) AS extract(HOUR FROM INTERVAL '123 12:34:56.789123' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select date_part('HOUR', interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +Project [date_part(HOUR, INTERVAL '123 12:34:56.789123' DAY TO SECOND) AS date_part(HOUR, INTERVAL '123 12:34:56.789123' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select extract(HOUR from -interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +Project [extract(HOUR, -INTERVAL '123 12:34:56.789123' DAY TO SECOND) AS extract(HOUR FROM (- INTERVAL '123 12:34:56.789123' DAY TO SECOND))#x] ++- OneRowRelation + + +-- !query +select extract(MINUTE from interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +Project [extract(MINUTE, INTERVAL '123 12:34:56.789123' DAY TO SECOND) AS extract(MINUTE FROM INTERVAL '123 12:34:56.789123' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select date_part('MINUTE', interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +Project [date_part(MINUTE, INTERVAL '123 12:34:56.789123' DAY TO SECOND) AS date_part(MINUTE, INTERVAL '123 12:34:56.789123' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select extract(MINUTE from -interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +Project [extract(MINUTE, -INTERVAL '123 12:34:56.789123' DAY TO SECOND) AS extract(MINUTE FROM (- INTERVAL '123 12:34:56.789123' DAY TO SECOND))#x] ++- OneRowRelation + + +-- !query +select extract(SECOND from interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +Project [extract(SECOND, INTERVAL '123 12:34:56.789123' DAY TO SECOND) AS extract(SECOND FROM INTERVAL '123 12:34:56.789123' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select date_part('SECOND', interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +Project [date_part(SECOND, INTERVAL '123 12:34:56.789123' DAY TO SECOND) AS date_part(SECOND, INTERVAL '123 12:34:56.789123' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select extract(SECOND from -interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +Project [extract(SECOND, -INTERVAL '123 12:34:56.789123' DAY TO SECOND) AS extract(SECOND FROM (- INTERVAL '123 12:34:56.789123' DAY TO SECOND))#x] ++- OneRowRelation + + +-- !query +select date_part(NULL, interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +Project [date_part(null, INTERVAL '123 12:34:56.789123' DAY TO SECOND) AS date_part(NULL, INTERVAL '123 12:34:56.789123' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select extract(MONTH from interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"INTERVAL '123 12:34:56.789123' DAY TO SECOND\"", + "field" : "`MONTH`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "extract(MONTH from interval '123 12:34:56.789123123' DAY TO SECOND)" + } ] +} + + +-- !query +select date_part('not_supported', interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"INTERVAL '123 12:34:56.789123' DAY TO SECOND\"", + "field" : "`not_supported`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "date_part('not_supported', interval '123 12:34:56.789123123' DAY TO SECOND)" + } ] +} + + +-- !query +select datepart('year', c), datepart('year', ntz), datepart('year', i) from t +-- !query analysis +Project [datepart(year, c#x) AS datepart(year FROM c)#x, datepart(year, ntz#x) AS datepart(year FROM ntz)#x, datepart(year, i#x) AS datepart(year FROM 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] + +- OneRowRelation + + +-- !query +select datepart('DAY', interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query analysis +Project [datepart(DAY, INTERVAL '123 12:34:56.789123' DAY TO SECOND) AS datepart(DAY FROM INTERVAL '123 12:34:56.789123' DAY TO SECOND)#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out new file mode 100644 index 0000000000000..a192a8741467c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out @@ -0,0 +1,746 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b) +-- !query analysis +CreateViewCommand `testData`, SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b), false, true, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH CUBE +-- !query analysis +Aggregate [(a#x + b#x)#x, b#x, spark_grouping_id#xL], [(a#x + b#x)#x AS (a + b)#x, b#x, sum((a#x - b#x)) AS sum((a - b))#xL] ++- Expand [[a#x, b#x, (a#x + b#x)#x, b#x, 0], [a#x, b#x, (a#x + b#x)#x, null, 1], [a#x, b#x, null, b#x, 2], [a#x, b#x, null, null, 3]], [a#x, b#x, (a#x + b#x)#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, (a#x + b#x) AS (a#x + b#x)#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH CUBE +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, sum(b#x) AS sum(b)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0], [a#x, b#x, a#x, null, 1], [a#x, b#x, null, b#x, 2], [a#x, b#x, null, null, 3]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP +-- !query analysis +Aggregate [(a#x + b#x)#x, b#x, spark_grouping_id#xL], [(a#x + b#x)#x AS (a + b)#x, b#x, sum((a#x - b#x)) AS sum((a - b))#xL] ++- Expand [[a#x, b#x, (a#x + b#x)#x, b#x, 0], [a#x, b#x, (a#x + b#x)#x, null, 1], [a#x, b#x, null, null, 3]], [a#x, b#x, (a#x + b#x)#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, (a#x + b#x) AS (a#x + b#x)#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH ROLLUP +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, sum(b#x) AS sum(b)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0], [a#x, b#x, a#x, null, 1], [a#x, b#x, null, null, 3]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW courseSales AS SELECT * FROM VALUES +("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000) +AS courseSales(course, year, earnings) +-- !query analysis +CreateViewCommand `courseSales`, SELECT * FROM VALUES +("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000) +AS courseSales(course, year, earnings), false, true, LocalTempView, true + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY course, year +-- !query analysis +Sort [course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true ++- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, sum(earnings#x) AS sum(earnings)#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year, (course, year)) ORDER BY course, year +-- !query analysis +Sort [course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true ++- Aggregate [course#x, year#x, spark_grouping_id#xL, _gen_grouping_pos#x], [course#x, year#x, sum(earnings#x) AS sum(earnings)#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0, 0], [course#x, year#x, earnings#x, course#x, year#x, 0, 1], [course#x, year#x, earnings#x, course#x, null, 1, 2], [course#x, year#x, earnings#x, null, null, 3, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year, (course, year), ()) ORDER BY course, year +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0051", + "messageParameters" : { + "element" : "ROLLUP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 53, + "stopIndex" : 101, + "fragment" : "GROUP BY ROLLUP(course, year, (course, year), ())" + } ] +} + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, year +-- !query analysis +Sort [course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true ++- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, sum(earnings#x) AS sum(earnings)#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year, (course, year)) ORDER BY course, year +-- !query analysis +Sort [course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true ++- Aggregate [course#x, year#x, spark_grouping_id#xL, _gen_grouping_pos#x], [course#x, year#x, sum(earnings#x) AS sum(earnings)#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0, 0], [course#x, year#x, earnings#x, course#x, year#x, 0, 1], [course#x, year#x, earnings#x, course#x, year#x, 0, 2], [course#x, year#x, earnings#x, course#x, null, 1, 3], [course#x, year#x, earnings#x, course#x, year#x, 0, 4], [course#x, year#x, earnings#x, null, year#x, 2, 5], [course#x, year#x, earnings#x, course#x, year#x, 0, 6], [course#x, year#x, earnings#x, null, null, 3, 7]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year, (course, year), ()) ORDER BY course, year +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0051", + "messageParameters" : { + "element" : "CUBE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 53, + "stopIndex" : 99, + "fragment" : "GROUP BY CUBE(course, year, (course, year), ())" + } ] +} + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year) +-- !query analysis +Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, sum(earnings#x) AS sum(earnings)#xL] ++- Expand [[course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year, ()) +-- !query analysis +Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, sum(earnings#x) AS sum(earnings)#xL] ++- Expand [[course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course) +-- !query analysis +Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, sum(earnings#x) AS sum(earnings)#xL] ++- Expand [[course#x, year#x, earnings#x, course#x, null, 1]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year) +-- !query analysis +Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, sum(earnings#x) AS sum(earnings)#xL] ++- Expand [[course#x, year#x, earnings#x, null, year#x, 2]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, CUBE(course, year) ORDER BY course, year +-- !query analysis +Sort [course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true ++- Aggregate [course#x, year#x, spark_grouping_id#xL, _gen_grouping_pos#x], [course#x, year#x, sum(earnings#x) AS sum(earnings)#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0, 0], [course#x, year#x, earnings#x, course#x, null, 1, 1], [course#x, year#x, earnings#x, course#x, year#x, 0, 2], [course#x, year#x, earnings#x, course#x, null, 1, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year), ROLLUP(course, year) ORDER BY course, year +-- !query analysis +Sort [course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true ++- Aggregate [course#x, year#x, spark_grouping_id#xL, _gen_grouping_pos#x], [course#x, year#x, sum(earnings#x) AS sum(earnings)#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0, 0], [course#x, year#x, earnings#x, course#x, year#x, 0, 1], [course#x, year#x, earnings#x, course#x, year#x, 0, 2], [course#x, year#x, earnings#x, course#x, year#x, 0, 3], [course#x, year#x, earnings#x, course#x, null, 1, 4], [course#x, year#x, earnings#x, course#x, null, 1, 5], [course#x, year#x, earnings#x, course#x, year#x, 0, 6], [course#x, year#x, earnings#x, course#x, year#x, 0, 7], [course#x, year#x, earnings#x, null, year#x, 2, 8], [course#x, year#x, earnings#x, course#x, year#x, 0, 9], [course#x, year#x, earnings#x, course#x, null, 1, 10], [course#x, year#x, earnings#x, null, null, 3, 11]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year), ROLLUP(course, year), GROUPING SETS(course, year) ORDER BY course, year +-- !query analysis +Sort [course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true ++- Aggregate [course#x, year#x, spark_grouping_id#xL, _gen_grouping_pos#x], [course#x, year#x, sum(earnings#x) AS sum(earnings)#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0, 0], [course#x, year#x, earnings#x, course#x, year#x, 0, 1], [course#x, year#x, earnings#x, course#x, year#x, 0, 2], [course#x, year#x, earnings#x, course#x, year#x, 0, 3], [course#x, year#x, earnings#x, course#x, year#x, 0, 4], [course#x, year#x, earnings#x, course#x, year#x, 0, 5], [course#x, year#x, earnings#x, course#x, year#x, 0, 6], [course#x, year#x, earnings#x, course#x, year#x, 0, 7], [course#x, year#x, earnings#x, course#x, null, 1, 8], [course#x, year#x, earnings#x, course#x, year#x, 0, 9], [course#x, year#x, earnings#x, course#x, null, 1, 10], [course#x, year#x, earnings#x, course#x, year#x, 0, 11], [course#x, year#x, earnings#x, course#x, year#x, 0, 12], [course#x, year#x, earnings#x, course#x, year#x, 0, 13], [course#x, year#x, earnings#x, course#x, year#x, 0, 14], [course#x, year#x, earnings#x, course#x, year#x, 0, 15], [course#x, year#x, earnings#x, course#x, year#x, 0, 16], [course#x, year#x, earnings#x, null, year#x, 2, 17], [course#x, year#x, earnings#x, course#x, year#x, 0, 18], [course#x, year#x, earnings#x, course#x, year#x, 0, 19], [course#x, year#x, earnings#x, course#x, null, 1, 20], [course#x, year#x, earnings#x, course#x, year#x, 0, 21], [course#x, year#x, earnings#x, course#x, null, 1, 22], [course#x, year#x, earnings#x, null, year#x, 2, 23]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, SUM(earnings) AS sum FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum +-- !query analysis +Sort [course#x ASC NULLS FIRST, sum#xL ASC NULLS FIRST], true ++- Aggregate [course#x, earnings#x, spark_grouping_id#xL], [course#x, sum(earnings#x) AS sum#xL] + +- Expand [[course#x, year#x, earnings#x, null, null, 3], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, course#x, earnings#x, 0]], [course#x, year#x, earnings#x, course#x, earnings#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, earnings#x AS earnings#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, SUM(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum +-- !query analysis +Sort [course#x ASC NULLS FIRST, sum#xL ASC NULLS FIRST], true ++- Aggregate [course#x, earnings#x, spark_grouping_id#xL], [course#x, sum(earnings#x) AS sum#xL, spark_grouping_id#xL AS grouping_id(course, earnings)#xL] + +- Expand [[course#x, year#x, earnings#x, null, null, 3], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, course#x, earnings#x, 0]], [course#x, year#x, earnings#x, course#x, earnings#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, earnings#x AS earnings#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales +GROUP BY CUBE(course, year) +-- !query analysis +Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(course)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(year)#x, spark_grouping_id#xL AS grouping_id(course, year)#xL] ++- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, GROUPING(course) FROM courseSales GROUP BY course, year +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2445", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 37, + "fragment" : "GROUPING(course)" + } ] +} + + +-- !query +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY course, year +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2407", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 46, + "fragment" : "GROUPING_ID(course, year)" + } ] +} + + +-- !query +SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, year +-- !query analysis +Sort [grouping__id#xL ASC NULLS FIRST, course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true ++- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL AS grouping__id#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) +HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, year +-- !query analysis +Sort [course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true ++- Project [course#x, year#x] + +- Filter ((cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) = 1) AND (spark_grouping_id#xL > cast(0 as bigint))) + +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL, spark_grouping_id#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING(course) > 0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION" +} + + +-- !query +SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION" +} + + +-- !query +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0 +-- !query analysis +Project [course#x, year#x] ++- Filter (spark_grouping_id#xL > cast(0 as bigint)) + +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, year +-- !query analysis +Project [course#x, year#x, grouping(course)#x, grouping(year)#x] ++- Sort [cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) ASC NULLS FIRST, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) ASC NULLS FIRST, course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true + +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(course)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(year)#x, spark_grouping_id#xL, spark_grouping_id#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, year +-- !query analysis +Project [course#x, year#x, grouping_id(course, year)#xL] ++- Sort [cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) ASC NULLS FIRST, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) ASC NULLS FIRST, course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true + +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL AS grouping_id(course, year)#xL, spark_grouping_id#xL, spark_grouping_id#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING(course) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION" +} + + +-- !query +SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING_ID(course) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION" +} + + +-- !query +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, year +-- !query analysis +Project [course#x, year#x] ++- Sort [spark_grouping_id#xL ASC NULLS FIRST, course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true + +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT a + b AS k1, b AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2) +-- !query analysis +Aggregate [k1#x, k2#x, spark_grouping_id#xL], [k1#x, k2#x, sum((a#x - b#x)) AS sum((a - b))#xL] ++- Expand [[a#x, b#x, k1#x, k2#x, 0], [a#x, b#x, k1#x, null, 1], [a#x, b#x, null, k2#x, 2], [a#x, b#x, null, null, 3]], [a#x, b#x, k1#x, k2#x, spark_grouping_id#xL] + +- Project [a#x, b#x, (a#x + b#x) AS k1#x, b#x AS k2#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a + b AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b) +-- !query analysis +Aggregate [k#x, b#x, spark_grouping_id#xL], [k#x, b#x, sum((a#x - b#x)) AS sum((a - b))#xL] ++- Expand [[a#x, b#x, k#x, b#x, 0], [a#x, b#x, k#x, null, 1], [a#x, b#x, null, null, 3]], [a#x, b#x, k#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, (a#x + b#x) AS k#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a + b, b AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k) +-- !query analysis +Aggregate [(a#x + b#x)#x, k#x, spark_grouping_id#xL], [(a#x + b#x)#x AS (a + b)#x, k#x, sum((a#x - b#x)) AS sum((a - b))#xL] ++- Expand [[a#x, b#x, null, k#x, 2]], [a#x, b#x, (a#x + b#x)#x, k#x, spark_grouping_id#xL] + +- Project [a#x, b#x, (a#x + b#x) AS (a#x + b#x)#x, b#x AS k#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, b, CUBE(a, b) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, b#x, 0, 1], [a#x, b#x, a#x, b#x, 0, 2], [a#x, b#x, a#x, b#x, 0, 3]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, b, ROLLUP(a, b) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, b#x, 0, 1], [a#x, b#x, a#x, b#x, 0, 2]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY CUBE(a, b), ROLLUP(a, b) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, b#x, 0, 1], [a#x, b#x, a#x, b#x, 0, 2], [a#x, b#x, a#x, b#x, 0, 3], [a#x, b#x, a#x, null, 1, 4], [a#x, b#x, a#x, null, 1, 5], [a#x, b#x, a#x, b#x, 0, 6], [a#x, b#x, a#x, b#x, 0, 7], [a#x, b#x, null, b#x, 2, 8], [a#x, b#x, a#x, b#x, 0, 9], [a#x, b#x, a#x, null, 1, 10], [a#x, b#x, null, null, 3, 11]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, CUBE(a, b), ROLLUP(b) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, b#x, 0, 1], [a#x, b#x, a#x, b#x, 0, 2], [a#x, b#x, a#x, null, 1, 3], [a#x, b#x, a#x, b#x, 0, 4], [a#x, b#x, a#x, b#x, 0, 5], [a#x, b#x, a#x, b#x, 0, 6], [a#x, b#x, a#x, null, 1, 7]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS((a, b), (a), ()) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, null, 1, 1], [a#x, b#x, a#x, null, 1, 2]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, CUBE(a, b), GROUPING SETS((a, b), (a), ()) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, b#x, 0, 1], [a#x, b#x, a#x, b#x, 0, 2], [a#x, b#x, a#x, b#x, 0, 3], [a#x, b#x, a#x, null, 1, 4], [a#x, b#x, a#x, null, 1, 5], [a#x, b#x, a#x, b#x, 0, 6], [a#x, b#x, a#x, b#x, 0, 7], [a#x, b#x, a#x, b#x, 0, 8], [a#x, b#x, a#x, b#x, 0, 9], [a#x, b#x, a#x, null, 1, 10], [a#x, b#x, a#x, null, 1, 11]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, CUBE(a, b), ROLLUP(a, b), GROUPING SETS((a, b), (a), ()) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, b#x, 0, 1], [a#x, b#x, a#x, b#x, 0, 2], [a#x, b#x, a#x, b#x, 0, 3], [a#x, b#x, a#x, b#x, 0, 4], [a#x, b#x, a#x, b#x, 0, 5], [a#x, b#x, a#x, b#x, 0, 6], [a#x, b#x, a#x, b#x, 0, 7], [a#x, b#x, a#x, b#x, 0, 8], [a#x, b#x, a#x, b#x, 0, 9], [a#x, b#x, a#x, b#x, 0, 10], [a#x, b#x, a#x, b#x, 0, 11], [a#x, b#x, a#x, b#x, 0, 12], [a#x, b#x, a#x, null, 1, 13], [a#x, b#x, a#x, null, 1, 14], [a#x, b#x, a#x, b#x, 0, 15], [a#x, b#x, a#x, null, 1, 16], [a#x, b#x, a#x, null, 1, 17], [a#x, b#x, a#x, b#x, 0, 18], [a#x, b#x, a#x, b#x, 0, 19], [a#x, b#x, a#x, b#x, 0, 20], [a#x, b#x, a#x, b#x, 0, 21], [a#x, b#x, a#x, b#x, 0, 22], [a#x, b#x, a#x, b#x, 0, 23], ... 12 more fields], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(ROLLUP(a, b)) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, null, 1, 1], [a#x, b#x, a#x, null, 1, 2]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b), (a), ())) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, null, 1, 1], [a#x, b#x, a#x, null, 1, 2]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS((a, b), GROUPING SETS(ROLLUP(a, b))) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, b#x, 0, 1], [a#x, b#x, a#x, null, 1, 2], [a#x, b#x, a#x, null, 1, 3]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS((a, b, a, b), (a, b, a), (a, b)) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, b#x, 0, 1], [a#x, b#x, a#x, b#x, 0, 2]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b, a, b), (a, b, a), (a, b))) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, b#x, 0, 1], [a#x, b#x, a#x, b#x, 0, 2]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(ROLLUP(a, b), CUBE(a, b)) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, null, 1, 1], [a#x, b#x, a#x, null, 1, 2], [a#x, b#x, a#x, b#x, 0, 3], [a#x, b#x, a#x, null, 1, 4], [a#x, b#x, a#x, b#x, 0, 5], [a#x, b#x, a#x, null, 1, 6]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b), (a), ()), GROUPING SETS((a, b), (a), (b), ())) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, null, 1, 1], [a#x, b#x, a#x, null, 1, 2], [a#x, b#x, a#x, b#x, 0, 3], [a#x, b#x, a#x, null, 1, 4], [a#x, b#x, a#x, b#x, 0, 5], [a#x, b#x, a#x, null, 1, 6]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS((a, b), (a), (), (a, b), (a), (b), ()) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, null, 1, 1], [a#x, b#x, a#x, null, 1, 2], [a#x, b#x, a#x, b#x, 0, 3], [a#x, b#x, a#x, null, 1, 4], [a#x, b#x, a#x, b#x, 0, 5], [a#x, b#x, a#x, null, 1, 6]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-duckdb.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-duckdb.sql.out new file mode 100644 index 0000000000000..c7420f4ad26e2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-duckdb.sql.out @@ -0,0 +1,144 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view integers as select * from values + (0, 1), + (0, 2), + (1, 3), + (1, NULL) + as integers(g, i) +-- !query analysis +CreateViewCommand `integers`, select * from values + (0, 1), + (0, 2), + (1, 3), + (1, NULL) + as integers(g, i), false, false, LocalTempView, true + +- Project [g#x, i#x] + +- SubqueryAlias integers + +- LocalRelation [g#x, i#x] + + +-- !query +SELECT g, SUM(i) FROM integers GROUP BY ALL ORDER BY 1 +-- !query analysis +Sort [g#x ASC NULLS FIRST], true ++- Aggregate [g#x], [g#x, sum(i#x) AS sum(i)#xL] + +- SubqueryAlias integers + +- View (`integers`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias integers + +- LocalRelation [g#x, i#x] + + +-- !query +SELECT g, SUM(i), COUNT(*), COUNT(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1 +-- !query analysis +Sort [g#x ASC NULLS FIRST], true ++- Aggregate [g#x], [g#x, sum(i#x) AS sum(i)#xL, count(1) AS count(1)#xL, count(i#x) AS count(i)#xL, sum(g#x) AS sum(g)#xL] + +- SubqueryAlias integers + +- View (`integers`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias integers + +- LocalRelation [g#x, i#x] + + +-- !query +SELECT i%2, SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1 +-- !query analysis +Sort [(i % 2)#x ASC NULLS FIRST], true ++- Aggregate [(i#x % 2)], [(i#x % 2) AS (i % 2)#x, sum(i#x) AS sum(i)#xL, sum(g#x) AS sum(g)#xL] + +- SubqueryAlias integers + +- View (`integers`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias integers + +- LocalRelation [g#x, i#x] + + +-- !query +SELECT (g+i)%2, SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1 +-- !query analysis +Sort [((g + i) % 2)#x ASC NULLS FIRST], true ++- Aggregate [((g#x + i#x) % 2)], [((g#x + i#x) % 2) AS ((g + i) % 2)#x, sum(i#x) AS sum(i)#xL, sum(g#x) AS sum(g)#xL] + +- SubqueryAlias integers + +- View (`integers`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias integers + +- LocalRelation [g#x, i#x] + + +-- !query +SELECT (g+i)%2 + SUM(i), SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 55, + "stopIndex" : 66, + "fragment" : "GROUP BY ALL" + } ] +} + + +-- !query +SELECT g, i, g%2, SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1, 2, 3, 4 +-- !query analysis +Sort [g#x ASC NULLS FIRST, i#x ASC NULLS FIRST, (g % 2)#x ASC NULLS FIRST, sum(i)#xL ASC NULLS FIRST], true ++- Aggregate [g#x, i#x, (g#x % 2)], [g#x, i#x, (g#x % 2) AS (g % 2)#x, sum(i#x) AS sum(i)#xL, sum(g#x) AS sum(g)#xL] + +- SubqueryAlias integers + +- View (`integers`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias integers + +- LocalRelation [g#x, i#x] + + +-- !query +SELECT c0 FROM (SELECT 1 c0) t0 GROUP BY ALL HAVING c0>0 +-- !query analysis +Filter (c0#x > 0) ++- Aggregate [c0#x], [c0#x] + +- SubqueryAlias t0 + +- Project [1 AS c0#x] + +- OneRowRelation + + +-- !query +SELECT c0 FROM (SELECT 1 c0, 1 c1 UNION ALL SELECT 1, 2) t0 GROUP BY ALL ORDER BY c0 +-- !query analysis +Sort [c0#x ASC NULLS FIRST], true ++- Aggregate [c0#x], [c0#x] + +- SubqueryAlias t0 + +- Union false, false + :- Project [1 AS c0#x, 1 AS c1#x] + : +- OneRowRelation + +- Project [1 AS 1#x, 2 AS 2#x] + +- OneRowRelation + + +-- !query +SELECT c0 FROM (SELECT 1 c0, 1 c1 UNION ALL SELECT 1, 2) t0 GROUP BY ALL HAVING c1>0 ORDER BY c0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`c1`", + "proposal" : "`t0`.`c0`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 81, + "stopIndex" : 82, + "fragment" : "c1" + } ] +} 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 new file mode 100644 index 0000000000000..a9fc06c1f523b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-mosha.sql.out @@ -0,0 +1,169 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view stuff as select * from values + (42, 9.75, 'hello world', '1970-08-07', '13.37', array(1,20,300)), + (1337, 1.2345, 'oh no', '2000-01-01', '42.0', array(4000,50000,600000)), + (42, 13.37, 'test', '1970-08-07', '1234567890', array(7000000,80000000,900000000)) + as stuff(i, f, s, t, d, a) +-- !query analysis +CreateViewCommand `stuff`, select * from values + (42, 9.75, 'hello world', '1970-08-07', '13.37', array(1,20,300)), + (1337, 1.2345, 'oh no', '2000-01-01', '42.0', array(4000,50000,600000)), + (42, 13.37, 'test', '1970-08-07', '1234567890', array(7000000,80000000,900000000)) + as stuff(i, f, s, t, d, a), false, false, LocalTempView, true + +- Project [i#x, f#x, s#x, t#x, d#x, a#x] + +- SubqueryAlias stuff + +- LocalRelation [i#x, f#x, s#x, t#x, d#x, a#x] + + +-- !query +SELECT 100 * SUM(i) + SUM(f) / COUNT(s) AS f1, i AS f2 FROM stuff GROUP BY ALL ORDER BY f2 +-- !query analysis +Sort [f2#x ASC NULLS FIRST], true ++- Aggregate [i#x], [(cast((cast(100 as bigint) * sum(i#x)) as decimal(20,0)) + (sum(f#x) / cast(count(s#x) as decimal(20,0)))) AS f1#x, i#x AS f2#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] + +- Project [i#x, f#x, s#x, t#x, d#x, a#x] + +- SubqueryAlias stuff + +- LocalRelation [i#x, f#x, s#x, t#x, d#x, a#x] + + +-- !query +SELECT i + 1 AS i1, COUNT(i - 2) ci, f / i AS fi, SUM(i + f) sif FROM stuff GROUP BY ALL ORDER BY 1, 3 +-- !query analysis +Sort [i1#x ASC NULLS FIRST, fi#x ASC NULLS FIRST], true ++- Aggregate [(i#x + 1), (f#x / cast(i#x as decimal(10,0)))], [(i#x + 1) AS i1#x, count((i#x - 2)) AS ci#xL, (f#x / cast(i#x as decimal(10,0))) AS fi#x, sum((cast(i#x as decimal(10,0)) + f#x)) AS sif#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] + +- Project [i#x, f#x, s#x, t#x, d#x, a#x] + +- SubqueryAlias stuff + +- LocalRelation [i#x, f#x, s#x, t#x, d#x, a#x] + + +-- !query +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 +-- !query analysis +Sort [i#x ASC NULLS FIRST, i#x ASC NULLS FIRST, ci#xL ASC NULLS FIRST, ci#xL ASC NULLS FIRST, f#x ASC NULLS FIRST, f#x ASC NULLS FIRST, sif#x ASC NULLS FIRST, sif#x ASC NULLS FIRST], true ++- Aggregate [i#x, f#x], [i#x AS i#x, count(i#x) AS ci#xL, f#x AS f#x, sum((cast(i#x as decimal(10,0)) + f#x)) AS sif#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] + +- Project [i#x, f#x, s#x, t#x, d#x, a#x] + +- SubqueryAlias stuff + +- LocalRelation [i#x, f#x, s#x, t#x, d#x, a#x] + + +-- !query +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 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] + +- 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] + +- Project [i#x, f#x, s#x, t#x, d#x, a#x] + +- SubqueryAlias stuff + +- LocalRelation [i#x, f#x, s#x, t#x, d#x, a#x] + + +-- !query +SELECT i + SUM(f) FROM stuff GROUP BY ALL +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 41, + "fragment" : "GROUP BY ALL" + } ] +} + + +-- !query +SELECT s AS s, COUNT(*) c FROM stuff GROUP BY ALL HAVING SUM(f) > 0 ORDER BY s +-- !query analysis +Sort [s#x ASC NULLS FIRST], true ++- Project [s#x, c#xL] + +- Filter (sum(f#x)#x > cast(cast(0 as decimal(1,0)) as decimal(16,4))) + +- Aggregate [s#x], [s#x AS s#x, count(1) AS c#xL, sum(f#x) AS sum(f#x)#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] + +- Project [i#x, f#x, s#x, t#x, d#x, a#x] + +- SubqueryAlias stuff + +- LocalRelation [i#x, f#x, s#x, t#x, d#x, a#x] + + +-- !query +SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING si > 2 +-- !query analysis +Filter (si#xL > cast(2 as bigint)) ++- Aggregate [sum(i#x) AS si#xL] + +- 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] + +- Project [i#x, f#x, s#x, t#x, d#x, a#x] + +- SubqueryAlias stuff + +- LocalRelation [i#x, f#x, s#x, t#x, d#x, a#x] + + +-- !query +SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING si < 2 +-- !query analysis +Filter (si#xL < cast(2 as bigint)) ++- Aggregate [sum(i#x) AS si#xL] + +- 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] + +- Project [i#x, f#x, s#x, t#x, d#x, a#x] + +- SubqueryAlias stuff + +- LocalRelation [i#x, f#x, s#x, t#x, d#x, a#x] + + +-- !query +SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING i > 2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`i`", + "proposal" : "`si`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 49, + "stopIndex" : 49, + "fragment" : "i" + } ] +} + + +-- !query +SELECT SUM(i) si FROM stuff GROUP BY ALL ORDER BY i DESC +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`i`", + "proposal" : "`si`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 51, + "fragment" : "i" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all.sql.out new file mode 100644 index 0000000000000..945a73495355a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all.sql.out @@ -0,0 +1,350 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view data as select * from values + ("USA", "San Francisco", "Reynold", 1, 11.0), + ("USA", "San Francisco", "Matei", 2, 12.0), + ("USA", "Berkeley", "Xiao", 3, 13.0), + ("China", "Hangzhou", "Wenchen", 4, 14.0), + ("China", "Shanghai", "Shanghaiese", 5, 15.0), + ("Korea", "Seoul", "Hyukjin", 6, 16.0), + ("UK", "London", "Sean", 7, 17.0) + as data(country, city, name, id, power) +-- !query analysis +CreateViewCommand `data`, select * from values + ("USA", "San Francisco", "Reynold", 1, 11.0), + ("USA", "San Francisco", "Matei", 2, 12.0), + ("USA", "Berkeley", "Xiao", 3, 13.0), + ("China", "Hangzhou", "Wenchen", 4, 14.0), + ("China", "Shanghai", "Shanghaiese", 5, 15.0), + ("Korea", "Seoul", "Hyukjin", 6, 16.0), + ("UK", "London", "Sean", 7, 17.0) + as data(country, city, name, id, power), false, false, LocalTempView, true + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select country, count(*) from data group by ALL +-- !query analysis +Aggregate [country#x], [country#x, count(1) AS count(1)#xL] ++- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select country, count(*) from data group by aLl +-- !query analysis +Aggregate [country#x], [country#x, count(1) AS count(1)#xL] ++- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select all, city, count(*) from (select country as all, city, id from data) group by all, city +-- !query analysis +Aggregate [all#x, city#x], [all#x, city#x, count(1) AS count(1)#xL] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [country#x AS all#x, city#x, id#x] + +- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +SELECT count(1) FROM VALUES(1), (2), (3) AS T(all) GROUP BY all +-- !query analysis +Aggregate [all#x], [count(1) AS count(1)#xL] ++- SubqueryAlias T + +- LocalRelation [all#x] + + +-- !query +select country, city, count(*), sum(power) from data group by all +-- !query analysis +Aggregate [country#x, city#x], [country#x, city#x, count(1) AS count(1)#xL, sum(power#x) AS sum(power)#x] ++- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select count(*), country, city, sum(power) from data group by all +-- !query analysis +Aggregate [country#x, city#x], [count(1) AS count(1)#xL, country#x, city#x, sum(power#x) AS sum(power)#x] ++- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select country as con, count(*) from data group by all +-- !query analysis +Aggregate [country#x], [country#x AS con#x, count(1) AS count(1)#xL] ++- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select country, count(*) as cnt from data group by all +-- !query analysis +Aggregate [country#x], [country#x, count(1) AS cnt#xL] ++- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select upper(country), count(*) as powerup from data group by all +-- !query analysis +Aggregate [upper(country#x)], [upper(country#x) AS upper(country)#x, count(1) AS powerup#xL] ++- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select country, sum(power) + 10 as powerup from data group by all +-- !query analysis +Aggregate [country#x], [country#x, (sum(power#x) + cast(10 as decimal(2,0))) AS powerup#x] ++- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select country, city from data group by all +-- !query analysis +Aggregate [country#x, city#x], [country#x, city#x] ++- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select con, powerup from + (select country as con, sum(power) + 10 as powerup from data group by all) +-- !query analysis +Project [con#x, powerup#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [country#x], [country#x AS con#x, (sum(power#x) + cast(10 as decimal(2,0))) AS powerup#x] + +- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select country, count(id) as cnt from data group by all having cnt > 1 +-- !query analysis +Filter (cnt#xL > cast(1 as bigint)) ++- Aggregate [country#x], [country#x, count(id#x) AS cnt#xL] + +- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select count(id) from data group by all +-- !query analysis +Aggregate [count(id#x) AS count(id)#xL] ++- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select count(id + power / 2) * 3 from data group by all +-- !query analysis +Aggregate [(count((cast(id#x as decimal(10,0)) + (power#x / cast(2 as decimal(1,0))))) * cast(3 as bigint)) AS (count((id + (power / 2))) * 3)#xL] ++- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select count(*) from (select * from data where country = "DNS") group by all +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [country#x, city#x, name#x, id#x, power#x] + +- Filter (country#x = DNS) + +- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select id + count(*) from data group by all +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 43, + "fragment" : "group by all" + } ] +} + + +-- !query +select (id + id) / 2 + count(*) * 2 from data group by all +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 47, + "stopIndex" : 58, + "fragment" : "group by all" + } ] +} + + +-- !query +select country, (select count(*) from data) as cnt, count(id) as cnt_id from data group by all +-- !query analysis +Aggregate [country#x, scalar-subquery#x []], [country#x, scalar-subquery#x [] AS cnt#xL, count(id#x) AS cnt_id#xL] +: :- Aggregate [count(1) AS count(1)#xL] +: : +- SubqueryAlias data +: : +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) +: : +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] +: : +- Project [country#x, city#x, name#x, id#x, power#x] +: : +- SubqueryAlias data +: : +- LocalRelation [country#x, city#x, name#x, id#x, power#x] +: +- Aggregate [count(1) AS count(1)#xL] +: +- SubqueryAlias data +: +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) +: +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] +: +- Project [country#x, city#x, name#x, id#x, power#x] +: +- SubqueryAlias data +: +- LocalRelation [country#x, city#x, name#x, id#x, power#x] ++- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select country, (select count(*) from data d1 where d1.country = d2.country), count(id) from data d2 group by all +-- !query analysis +Aggregate [country#x, scalar-subquery#x [country#x]], [country#x, scalar-subquery#x [country#x] AS scalarsubquery(country)#xL, count(id#x) AS count(id)#xL] +: :- Aggregate [count(1) AS count(1)#xL] +: : +- Filter (country#x = outer(country#x)) +: : +- SubqueryAlias d1 +: : +- SubqueryAlias data +: : +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) +: : +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] +: : +- Project [country#x, city#x, name#x, id#x, power#x] +: : +- SubqueryAlias data +: : +- LocalRelation [country#x, city#x, name#x, id#x, power#x] +: +- Aggregate [count(1) AS count(1)#xL] +: +- Filter (country#x = outer(country#x)) +: +- SubqueryAlias d1 +: +- SubqueryAlias data +: +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) +: +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] +: +- Project [country#x, city#x, name#x, id#x, power#x] +: +- SubqueryAlias data +: +- LocalRelation [country#x, city#x, name#x, id#x, power#x] ++- SubqueryAlias d2 + +- SubqueryAlias data + +- View (`data`, [country#x,city#x,name#x,id#x,power#x]) + +- Project [cast(country#x as string) AS country#x, cast(city#x as string) AS city#x, cast(name#x as string) AS name#x, cast(id#x as int) AS id#x, cast(power#x as decimal(3,1)) AS power#x] + +- Project [country#x, city#x, name#x, id#x, power#x] + +- SubqueryAlias data + +- LocalRelation [country#x, city#x, name#x, id#x, power#x] + + +-- !query +select (select count(*) from data d1 where d1.country = d2.country) + count(id) from data d2 group by all +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 94, + "stopIndex" : 105, + "fragment" : "group by all" + } ] +} + + +-- !query +select non_exist from data group by all +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`non_exist`", + "proposal" : "`data`.`city`, `data`.`id`, `data`.`name`, `data`.`power`, `data`.`country`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "non_exist" + } ] +} 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 new file mode 100644 index 0000000000000..bfb11e1e88e96 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-filter.sql.out @@ -0,0 +1,958 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query analysis +CreateViewCommand `testData`, SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b), false, true, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query analysis +CreateViewCommand `EMP`, SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id), false, true, LocalTempView, true + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query analysis +CreateViewCommand `DEPT`, SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state), false, true, LocalTempView, true + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData" + } ] +} + + +-- !query +SELECT COUNT(a) FILTER (WHERE a = 1), COUNT(b) FILTER (WHERE a > 1) FROM testData +-- !query analysis +Aggregate [count(a#x) FILTER (WHERE (a#x = 1)) AS count(a) FILTER (WHERE (a = 1))#xL, count(b#x) FILTER (WHERE (a#x > 1)) AS count(b) FILTER (WHERE (a > 1))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT COUNT(id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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)))) 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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !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] ++- 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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT COUNT(id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") = "2001-01-01") FROM emp +-- !query analysis +Aggregate [count(id#x) FILTER (WHERE (date_format(cast(hiredate#x as timestamp), yyyy-MM-dd, Some(America/Los_Angeles)) = 2001-01-01)) AS count(id) FILTER (WHERE (date_format(hiredate, yyyy-MM-dd) = 2001-01-01))#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp +-- !query analysis +Aggregate [count(distinct id#x) FILTER (WHERE (date_format(cast(hiredate#x as timestamp), yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles)) = 2001-01-01 00:00:00)) AS count(DISTINCT id) FILTER (WHERE (date_format(hiredate, yyyy-MM-dd HH:mm:ss) = 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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp +-- !query analysis +Aggregate [count(distinct id#x) AS count(DISTINCT id)#xL, count(distinct id#x) FILTER (WHERE (date_format(cast(hiredate#x as timestamp), yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles)) = 2001-01-01 00:00:00)) AS count(DISTINCT id) FILTER (WHERE (date_format(hiredate, yyyy-MM-dd HH:mm:ss) = 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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !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)))) 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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT SUM(salary), COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData +-- !query analysis +Aggregate [count(distinct 1) FILTER (WHERE (a#x = 1)) AS count(DISTINCT 1) FILTER (WHERE (a = 1))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE true) FROM emp +-- !query analysis +Aggregate [count(distinct id#x) FILTER (WHERE true) AS count(DISTINCT id) FILTER (WHERE true)#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE false) FROM emp +-- !query analysis +Aggregate [count(distinct id#x) FILTER (WHERE false) AS count(DISTINCT id) FILTER (WHERE false)#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id = 40) FROM emp +-- !query analysis +Aggregate [count(distinct 2) AS count(DISTINCT 2)#xL, count(distinct 2, 3) FILTER (WHERE (dept_id#x = 40)) AS count(DISTINCT 2, 3) FILTER (WHERE (dept_id = 40))#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id = 40) FROM emp +-- !query analysis +Aggregate [count(distinct 2) AS count(DISTINCT 2)#xL, count(distinct 3, 2) FILTER (WHERE (dept_id#x = 40)) AS count(DISTINCT 3, 2) FILTER (WHERE (dept_id = 40))#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id > 0) FROM emp +-- !query analysis +Aggregate [count(distinct 2) AS count(DISTINCT 2)#xL, count(distinct 2, 3) FILTER (WHERE (dept_id#x > 0)) AS count(DISTINCT 2, 3) FILTER (WHERE (dept_id > 0))#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id > 0) FROM emp +-- !query analysis +Aggregate [count(distinct 2) AS count(DISTINCT 2)#xL, count(distinct 3, 2) FILTER (WHERE (dept_id#x > 0)) AS count(DISTINCT 3, 2) FILTER (WHERE (dept_id > 0))#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id = 40) FROM emp +-- !query analysis +Aggregate [count(distinct id#x) AS count(DISTINCT id)#xL, count(distinct 2, 3) FILTER (WHERE (dept_id#x = 40)) AS count(DISTINCT 2, 3) FILTER (WHERE (dept_id = 40))#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id = 40) FROM emp +-- !query analysis +Aggregate [count(distinct id#x) AS count(DISTINCT id)#xL, count(distinct 3, 2) FILTER (WHERE (dept_id#x = 40)) AS count(DISTINCT 3, 2) FILTER (WHERE (dept_id = 40))#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id > 0) FROM emp +-- !query analysis +Aggregate [count(distinct id#x) AS count(DISTINCT id)#xL, count(distinct 2, 3) FILTER (WHERE (dept_id#x > 0)) AS count(DISTINCT 2, 3) FILTER (WHERE (dept_id > 0))#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id > 0) FROM emp +-- !query analysis +Aggregate [count(distinct id#x) AS count(DISTINCT id)#xL, count(distinct 3, 2) FILTER (WHERE (dept_id#x > 0)) AS count(DISTINCT 3, 2) FILTER (WHERE (dept_id > 0))#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData GROUP BY a +-- !query analysis +Aggregate [a#x], [a#x, count(b#x) FILTER (WHERE (a#x >= 2)) AS count(b) FILTER (WHERE (a >= 2))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, COUNT(b) FILTER (WHERE a != 2) FROM testData GROUP BY b +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT COUNT(a) FILTER (WHERE a >= 0), COUNT(b) FILTER (WHERE a >= 3) FROM testData GROUP BY a +-- !query analysis +Aggregate [a#x], [count(a#x) FILTER (WHERE (a#x >= 0)) AS count(a) FILTER (WHERE (a >= 0))#xL, count(b#x) FILTER (WHERE (a#x >= 3)) AS count(b) FILTER (WHERE (a >= 3))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > date "2003-01-01") FROM emp GROUP BY dept_id +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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)))) 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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !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] ++- 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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT dept_id, SUM(salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2003-01-01") FROM emp GROUP BY dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, sum(salary#x) FILTER (WHERE (date_format(cast(hiredate#x as timestamp), yyyy-MM-dd, Some(America/Los_Angeles)) > 2003-01-01)) AS sum(salary) FILTER (WHERE (date_format(hiredate, yyyy-MM-dd) > 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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, sum(distinct salary#x) FILTER (WHERE (date_format(cast(hiredate#x as timestamp), yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles)) > 2001-01-01 00:00:00)) AS sum(DISTINCT salary) FILTER (WHERE (date_format(hiredate, yyyy-MM-dd HH:mm:ss) > 2001-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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT dept_id, SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, sum(distinct salary#x) AS sum(DISTINCT salary)#x, sum(distinct salary#x) FILTER (WHERE (date_format(cast(hiredate#x as timestamp), yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles)) > 2001-01-01 00:00:00)) AS sum(DISTINCT salary) FILTER (WHERE (date_format(hiredate, yyyy-MM-dd HH:mm:ss) > 2001-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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE hiredate > date "2001-01-01"), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT dept_id, COUNT(id), SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2001-01-01") FROM emp GROUP BY dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(id#x) AS count(id)#xL, sum(distinct salary#x) AS sum(DISTINCT salary)#x, sum(distinct salary#x) FILTER (WHERE (date_format(cast(hiredate#x as timestamp), yyyy-MM-dd, Some(America/Los_Angeles)) > 2001-01-01)) AS sum(DISTINCT salary) FILTER (WHERE (date_format(hiredate, yyyy-MM-dd) > 2001-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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT b, COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData GROUP BY b +-- !query analysis +Aggregate [b#x], [b#x, count(distinct 1) FILTER (WHERE (a#x = 1)) AS count(DISTINCT 1) FILTER (WHERE (a = 1))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT 'foo', COUNT(a) FILTER (WHERE b <= 2) FROM testData GROUP BY 1 +-- !query analysis +Aggregate [foo], [foo AS foo#x, count(a#x) FILTER (WHERE (b#x <= 2)) AS count(a) FILTER (WHERE (b <= 2))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= date "2003-01-01") FROM emp GROUP BY 1 +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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)))) 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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !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] ++- 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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(distinct emp_name#x) AS count(DISTINCT emp_name)#xL, count(distinct hiredate#x) AS count(DISTINCT hiredate)#xL, sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE (id#x > 200)) AS sum(salary) FILTER (WHERE (id > 200))#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(distinct emp_name#x) AS count(DISTINCT emp_name)#xL, count(distinct hiredate#x) AS count(DISTINCT hiredate)#xL, sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE ((id#x + dept_id#x) > 500)) AS sum(salary) FILTER (WHERE ((id + dept_id) > 500))#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(distinct emp_name#x) AS count(DISTINCT emp_name)#xL, count(distinct hiredate#x) AS count(DISTINCT hiredate)#xL, sum(salary#x) FILTER (WHERE (salary#x < 400.0)) AS sum(salary) FILTER (WHERE (salary < 400.0))#x, sum(salary#x) FILTER (WHERE (id#x > 200)) AS sum(salary) FILTER (WHERE (id > 200))#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(distinct emp_name#x) AS count(DISTINCT emp_name)#xL, count(distinct hiredate#x) AS count(DISTINCT hiredate)#xL, sum(salary#x) FILTER (WHERE (salary#x < 400.0)) AS sum(salary) FILTER (WHERE (salary < 400.0))#x, sum(salary#x) FILTER (WHERE ((id#x + dept_id#x) > 500)) AS sum(salary) FILTER (WHERE ((id + dept_id) > 500))#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(distinct emp_name#x) FILTER (WHERE (id#x > 200)) AS count(DISTINCT emp_name) FILTER (WHERE (id > 200))#xL, sum(salary#x) AS sum(salary)#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(distinct emp_name#x) FILTER (WHERE ((id#x + dept_id#x) > 500)) AS count(DISTINCT emp_name) FILTER (WHERE ((id + dept_id) > 500))#xL, sum(salary#x) AS sum(salary)#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(distinct emp_name#x) AS count(DISTINCT emp_name)#xL, count(distinct emp_name#x) FILTER (WHERE (id#x > 200)) AS count(DISTINCT emp_name) FILTER (WHERE (id > 200))#xL, sum(salary#x) AS sum(salary)#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(distinct emp_name#x) AS count(DISTINCT emp_name)#xL, count(distinct emp_name#x) FILTER (WHERE ((id#x + dept_id#x) > 500)) AS count(DISTINCT emp_name) FILTER (WHERE ((id + dept_id) > 500))#xL, sum(salary#x) AS sum(salary)#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(distinct emp_name#x) AS count(DISTINCT emp_name)#xL, count(distinct emp_name#x) FILTER (WHERE (id#x > 200)) AS count(DISTINCT emp_name) FILTER (WHERE (id > 200))#xL, sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE (id#x > 200)) AS sum(salary) FILTER (WHERE (id > 200))#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(distinct emp_name#x) AS count(DISTINCT emp_name)#xL, count(distinct emp_name#x) FILTER (WHERE ((id#x + dept_id#x) > 500)) AS count(DISTINCT emp_name) FILTER (WHERE ((id + dept_id) > 500))#xL, sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE (id#x > 200)) AS sum(salary) FILTER (WHERE (id > 200))#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(distinct emp_name#x) FILTER (WHERE (id#x > 200)) AS count(DISTINCT emp_name) FILTER (WHERE (id > 200))#xL, count(distinct hiredate#x) AS count(DISTINCT hiredate)#xL, sum(salary#x) AS sum(salary)#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name), sum(salary) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(distinct emp_name#x) FILTER (WHERE (id#x > 200)) AS count(DISTINCT emp_name) FILTER (WHERE (id > 200))#xL, count(distinct emp_name#x) AS count(DISTINCT emp_name)#xL, sum(salary#x) AS sum(salary)#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, sum(distinct (id#x + dept_id#x)) FILTER (WHERE (id#x > 200)) AS sum(DISTINCT (id + dept_id)) FILTER (WHERE (id > 200))#xL, count(distinct hiredate#x) AS count(DISTINCT hiredate)#xL, sum(salary#x) AS sum(salary)#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dept_id, avg(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select dept_id, count(distinct emp_name, hiredate) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(distinct emp_name#x, hiredate#x) FILTER (WHERE (id#x > 200)) AS count(DISTINCT emp_name, hiredate) FILTER (WHERE (id > 200))#xL, sum(salary#x) AS sum(salary)#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, count(distinct emp_name, hiredate) filter (where id > 0), sum(salary) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(distinct emp_name#x, hiredate#x) FILTER (WHERE (id#x > 0)) AS count(DISTINCT emp_name, hiredate) FILTER (WHERE (id > 0))#xL, sum(salary#x) AS sum(salary)#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +select dept_id, count(distinct 1), count(distinct 1) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, count(distinct 1) AS count(DISTINCT 1)#xL, count(distinct 1) FILTER (WHERE (id#x > 200)) AS count(DISTINCT 1) FILTER (WHERE (id > 200))#xL, sum(salary#x) AS sum(salary)#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FILTER (WHERE b >= 0) FROM testData WHERE a = 0 GROUP BY 1 +-- !query analysis +Aggregate [foo], [foo AS foo#x, approx_count_distinct(a#x, 0.05, 0, 0) FILTER (WHERE (b#x >= 0)) AS approx_count_distinct(a) FILTER (WHERE (b >= 0))#xL] ++- Filter (a#x = 0) + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT 'foo', MAX(STRUCT(a)) FILTER (WHERE b >= 1) FROM testData WHERE a = 0 GROUP BY 1 +-- !query analysis +Aggregate [foo], [foo AS foo#x, max(struct(a, a#x)) FILTER (WHERE (b#x >= 1)) AS max(struct(a)) FILTER (WHERE (b >= 1))#x] ++- Filter (a#x = 0) + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a + b, COUNT(b) FILTER (WHERE b >= 2) FROM testData GROUP BY a + b +-- !query analysis +Aggregate [(a#x + b#x)], [(a#x + b#x) AS (a + b)#x, count(b#x) FILTER (WHERE (b#x >= 2)) AS count(b) FILTER (WHERE (b >= 2))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a + 2, COUNT(b) FILTER (WHERE b IN (1, 2)) FROM testData GROUP BY a + 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT a + 1 + 1, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY a + 1 +-- !query analysis +Aggregate [(a#x + 1)], [((a#x + 1) + 1) AS ((a + 1) + 1)#x, count(b#x) FILTER (WHERE (b#x > 0)) AS count(b) FILTER (WHERE (b > 0))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT COUNT(DISTINCT b) FILTER (WHERE b > 0), COUNT(DISTINCT b, c) FILTER (WHERE b > 0 AND c > 2) +FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +-- !query analysis +Aggregate [a#x], [count(distinct b#x) FILTER (WHERE (b#x > 0)) AS count(DISTINCT b) FILTER (WHERE (b > 0))#xL, count(distinct b#x, c#x) FILTER (WHERE ((b#x > 0) AND (c#x > 2))) AS count(DISTINCT b, c) FILTER (WHERE ((b > 0) AND (c > 2)))#xL] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [1 AS a#x, 2 AS b#x, 3 AS c#x] + +- OneRowRelation + + +-- !query +SELECT a AS k, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY k +-- !query analysis +Aggregate [a#x], [a#x AS k#x, count(b#x) FILTER (WHERE (b#x > 0)) AS count(b) FILTER (WHERE (b > 0))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT emp.dept_id, + avg(salary), + avg(salary) FILTER (WHERE id > (SELECT 200)) +FROM emp +GROUP BY dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, avg(salary#x) FILTER (WHERE (id#x > scalar-subquery#x [])) AS avg(salary) FILTER (WHERE (id > scalarsubquery()))#x] +: +- Project [200 AS 200#x] +: +- OneRowRelation ++- 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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT emp.dept_id, + avg(salary), + avg(salary) FILTER (WHERE emp.dept_id = (SELECT dept_id FROM dept LIMIT 1)) +FROM emp +GROUP BY dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, avg(salary#x) FILTER (WHERE (dept_id#x = scalar-subquery#x [])) AS avg(salary) FILTER (WHERE (dept_id = scalarsubquery()))#x] +: +- GlobalLimit 1 +: +- LocalLimit 1 +: +- Project [dept_id#x] +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT emp.dept_id, + avg(salary), + avg(salary) FILTER (WHERE EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id)) +FROM emp +GROUP BY dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, avg(salary#x) FILTER (WHERE exists#x [dept_id#x]) AS avg(salary) FILTER (WHERE exists(dept_id))#x] +: +- Project [state#x] +: +- Filter (dept_id#x = outer(dept_id#x)) +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT emp.dept_id, + Sum(salary), + Sum(salary) FILTER (WHERE NOT EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id)) +FROM emp +GROUP BY dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE NOT exists#x [dept_id#x]) AS sum(salary) FILTER (WHERE (NOT exists(dept_id)))#x] +: +- Project [state#x] +: +- Filter (dept_id#x = outer(dept_id#x)) +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT emp.dept_id, + avg(salary), + avg(salary) FILTER (WHERE emp.dept_id IN (SELECT DISTINCT dept_id + FROM dept)) +FROM emp +GROUP BY dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, avg(salary#x) FILTER (WHERE dept_id#x IN (list#x [])) AS avg(salary) FILTER (WHERE (dept_id IN (listquery())))#x] +: +- Distinct +: +- Project [dept_id#x] +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT emp.dept_id, + Sum(salary), + Sum(salary) FILTER (WHERE emp.dept_id NOT IN (SELECT DISTINCT dept_id + FROM dept)) +FROM emp +GROUP BY dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE NOT dept_id#x IN (list#x [])) AS sum(salary) FILTER (WHERE (NOT (dept_id IN (listquery()))))#x] +: +- Distinct +: +- Project [dept_id#x] +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT t1.b FROM (SELECT COUNT(b) FILTER (WHERE a >= 2) AS b FROM testData) t1 +-- !query analysis +Project [b#xL] ++- SubqueryAlias t1 + +- Aggregate [count(b#x) FILTER (WHERE (a#x >= 2)) AS b#xL] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-ordinal.sql.out new file mode 100644 index 0000000000000..c8c34a856d492 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-ordinal.sql.out @@ -0,0 +1,489 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b) +-- !query analysis +CreateViewCommand `data`, select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b), false, false, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a, sum(b) from data group by 1 +-- !query analysis +Aggregate [a#x], [a#x, sum(b#x) AS sum(b)#xL] ++- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select 1, 2, sum(b) from data group by 1, 2 +-- !query analysis +Aggregate [1, 2], [1 AS 1#x, 2 AS 2#x, sum(b#x) AS sum(b)#xL] ++- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a, 1, sum(b) from data group by a, 1 +-- !query analysis +Aggregate [a#x, a#x], [a#x, 1 AS 1#x, sum(b#x) AS sum(b)#xL] ++- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a, 1, sum(b) from data group by 1, 2 +-- !query analysis +Aggregate [a#x, 1], [a#x, 1 AS 1#x, sum(b#x) AS sum(b)#xL] ++- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a, b + 2, count(2) from data group by a, 2 +-- !query analysis +Aggregate [a#x, (b#x + 2)], [a#x, (b#x + 2) AS (b + 2)#x, count(2) AS count(2)#xL] ++- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a as aa, b + 2 as bb, count(2) from data group by 1, 2 +-- !query analysis +Aggregate [a#x, (b#x + 2)], [a#x AS aa#x, (b#x + 2) AS bb#x, count(2) AS count(2)#xL] ++- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select sum(b) from data group by 1 + 0 +-- !query analysis +Aggregate [(1 + 0)], [sum(b#x) AS sum(b)#xL] ++- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a, b from data group by -1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 33, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b from data group by 0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "0", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 32, + "fragment" : "0" + } ] +} + + +-- !query +select a, b from data group by 3 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "3", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 32, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, sum(b) from data group by 3 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "sum(data.b) AS `sum(b)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 40, + "stopIndex" : 40, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, sum(b) + 2 from data group by 3 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "(sum(data.b) + CAST(2 AS BIGINT)) AS `(sum(b) + 2)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 44, + "fragment" : "3" + } ] +} + + +-- !query +select a, rand(0), sum(b) +from +(select /*+ REPARTITION(1) */ a, b from data) group by a, 2 +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select * from data group by a, b, 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "STAR_GROUP_BY_POS", + "sqlState" : "0A000" +} + + +-- !query +select a, count(a) from (select 1 as a) tmp group by 1 order by 1 +-- !query analysis +Sort [a#x ASC NULLS FIRST], true ++- Aggregate [a#x], [a#x, count(a#x) AS count(a)#xL] + +- SubqueryAlias tmp + +- Project [1 AS a#x] + +- OneRowRelation + + +-- !query +select count(a), a from (select 1 as a) tmp group by 2 having a > 0 +-- !query analysis +Filter (a#x > 0) ++- Aggregate [a#x], [count(a#x) AS count(a)#xL, a#x] + +- SubqueryAlias tmp + +- Project [1 AS a#x] + +- OneRowRelation + + +-- !query +select a, a AS k, count(b) from data group by k, 1 +-- !query analysis +Aggregate [a#x, a#x], [a#x, a#x AS k#x, count(b#x) AS count(b)#xL] ++- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a, b, count(1) from data group by cube(1, 2) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0], [a#x, b#x, a#x, null, 1], [a#x, b#x, null, b#x, 2], [a#x, b#x, null, null, 3]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a, b, count(1) from data group by cube(1, b) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0], [a#x, b#x, a#x, null, 1], [a#x, b#x, null, b#x, 2], [a#x, b#x, null, null, 3]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a, b, count(1) from data group by 1, 2 with cube +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0], [a#x, b#x, a#x, null, 1], [a#x, b#x, null, b#x, 2], [a#x, b#x, null, null, 3]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a, b, count(1) from data group by rollup(1, 2) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0], [a#x, b#x, a#x, null, 1], [a#x, b#x, null, null, 3]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a, b, count(1) from data group by rollup(1, b) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0], [a#x, b#x, a#x, null, 1], [a#x, b#x, null, null, 3]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a, b, count(1) from data group by 1, 2 with rollup +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, b#x, 0], [a#x, b#x, a#x, null, 1], [a#x, b#x, null, null, 3]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a, b, count(1) from data group by grouping sets((1), (2), (1, 2)) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, null, 1], [a#x, b#x, null, b#x, 2], [a#x, b#x, a#x, b#x, 0]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a, b, count(1) from data group by grouping sets((1), (b), (a, 2)) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, null, 1], [a#x, b#x, null, b#x, 2], [a#x, b#x, a#x, b#x, 0]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2)) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, a#x, null, 1], [a#x, b#x, null, b#x, 2], [a#x, b#x, a#x, b#x, 0]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a, b, count(1) from data group by a, -1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 46, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b, count(1) from data group by a, 3 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "count(1) AS `count(1)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 45, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, count(1) from data group by cube(-1, 2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 47, + "stopIndex" : 48, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b, count(1) from data group by cube(1, 3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "count(1) AS `count(1)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 50, + "stopIndex" : 50, + "fragment" : "3" + } ] +} + + +-- !query +set spark.sql.groupByOrdinal=false +-- !query analysis +SetCommand (spark.sql.groupByOrdinal,Some(false)) + + +-- !query +select sum(b) from data group by -1 +-- !query analysis +Aggregate [-1], [sum(b#x) AS sum(b)#xL] ++- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#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 new file mode 100644 index 0000000000000..c65587010d2a6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by.sql.out @@ -0,0 +1,1177 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query analysis +CreateViewCommand `testData`, SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b), false, true, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, COUNT(b) FROM testData +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 32, + "fragment" : "SELECT a, COUNT(b) FROM testData" + } ] +} + + +-- !query +SELECT COUNT(a), COUNT(b) FROM testData +-- !query analysis +Aggregate [count(a#x) AS count(a)#xL, count(b#x) AS count(b)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, COUNT(b) FROM testData GROUP BY a +-- !query analysis +Aggregate [a#x], [a#x, count(b#x) AS count(b)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, COUNT(b) FROM testData GROUP BY b +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a +-- !query analysis +Aggregate [a#x], [count(a#x) AS count(a)#xL, count(b#x) AS count(b)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT 'foo', COUNT(a) FROM testData GROUP BY 1 +-- !query analysis +Aggregate [foo], [foo AS foo#x, count(a#x) AS count(a)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1 +-- !query analysis +Aggregate [foo], [foo AS foo#x] ++- Filter (a#x = 0) + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1 +-- !query analysis +Aggregate [foo], [foo AS foo#x, approx_count_distinct(a#x, 0.05, 0, 0) AS approx_count_distinct(a)#xL] ++- Filter (a#x = 0) + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1 +-- !query analysis +Aggregate [foo], [foo AS foo#x, max(struct(a, a#x)) AS max(struct(a))#x] ++- Filter (a#x = 0) + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a + b, COUNT(b) FROM testData GROUP BY a + b +-- !query analysis +Aggregate [(a#x + b#x)], [(a#x + b#x) AS (a + b)#x, count(b#x) AS count(b)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1 +-- !query analysis +Aggregate [(a#x + 1)], [((a#x + 1) + 1) AS ((a + 1) + 1)#x, count(b#x) AS count(b)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT count(1) FROM testData GROUP BY struct(a + 0.1 AS aa) +-- !query analysis +Aggregate [struct(aa, (cast(a#x as decimal(10,0)) + 0.1))], [count(1) AS count(1)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +FROM testData +-- !query analysis +Aggregate [skewness(cast(a#x as double)) AS skewness(a)#x, kurtosis(cast(a#x as double)) AS kurtosis(a)#x, min(a#x) AS min(a)#x, max(a#x) AS max(a)#x, avg(a#x) AS avg(a)#x, variance(cast(a#x as double)) AS variance(a)#x, stddev(cast(a#x as double)) AS stddev(a)#x, sum(a#x) AS sum(a)#xL, count(a#x) AS count(a)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +-- !query analysis +Aggregate [a#x], [count(distinct b#x) AS count(DISTINCT b)#xL, count(distinct b#x, c#x) AS count(DISTINCT b, c)#xL] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [1 AS a#x, 2 AS b#x, 3 AS c#x] + +- OneRowRelation + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k +-- !query analysis +Aggregate [a#x], [a#x AS k#x, count(b#x) AS count(b)#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1 +-- !query analysis +Filter (k#x > 1) ++- Aggregate [a#x], [a#x AS k#x, count(b#x) AS count(b)#xL] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`non_existing`", + "proposal" : "`testdata`.`a`, `testdata`.`b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 33, + "fragment" : "non_existing" + } ] +} + + +-- !query +SELECT COUNT(b) AS k FROM testData GROUP BY k +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "sqlExpr" : "count(testdata.b)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "COUNT(b)" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) +-- !query analysis +CreateViewCommand `testDataHasSameNameWithAlias`, SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v), false, true, LocalTempView, true + +- Project [k#x, a#x, v#x] + +- SubqueryAlias testDataHasSameNameWithAlias + +- LocalRelation [k#x, a#x, v#x] + + +-- !query +SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"k\"", + "expressionAnyValue" : "\"any_value(k)\"" + } +} + + +-- !query +set spark.sql.groupByAliases=false +-- !query analysis +SetCommand (spark.sql.groupByAliases,Some(false)) + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`k`", + "proposal" : "`testdata`.`a`, `testdata`.`b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 48, + "stopIndex" : 48, + "fragment" : "k" + } ] +} + + +-- !query +SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a +-- !query analysis +Aggregate [a#x], [a#x, count(1) AS count(1)#xL] ++- Filter false + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT COUNT(1) FROM testData WHERE false +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter false + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t +-- !query analysis +Project [1 AS 1#x] ++- SubqueryAlias t + +- Aggregate [count(1) AS count(1)#xL] + +- Filter false + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT 1 from ( + SELECT 1 AS z, + MIN(a.x) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z +-- !query analysis +Project [1 AS 1#x] ++- Filter NOT (z#x = z#x) + +- SubqueryAlias b + +- Aggregate [1 AS z#x, min(x#x) AS min(x)#x] + +- Filter false + +- SubqueryAlias a + +- Project [1 AS x#x] + +- OneRowRelation + + +-- !query +SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) +-- !query analysis +Aggregate [corr(distinct cast(x#x as double), cast(y#x as double)) AS corr(DISTINCT x, y)#x, corr(distinct cast(y#x as double), cast(x#x as double)) AS corr(DISTINCT y, x)#x, count(1) AS count(1)#xL] ++- SubqueryAlias t + +- Project [col1#x AS x#x, col2#x AS y#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT 1 FROM range(10) HAVING true +-- !query analysis +Filter cast(true as boolean) ++- Aggregate [1 AS 1#x] + +- Range (0, 10, step=1, splits=None) + + +-- !query +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query analysis +Project [1#x] ++- Filter (max(id#xL)#xL > cast(0 as bigint)) + +- Aggregate [1 AS 1#x, max(id#xL) AS max(id#xL)#xL] + +- Range (0, 10, step=1, splits=None) + + +-- !query +SELECT id FROM range(10) HAVING id > 0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 38, + "fragment" : "SELECT id FROM range(10) HAVING id > 0" + } ] +} + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true +-- !query analysis +SetCommand (spark.sql.legacy.parser.havingWithoutGroupByAsWhere,Some(true)) + + +-- !query +SELECT 1 FROM range(10) HAVING true +-- !query analysis +Filter cast(true as boolean) ++- Project [1 AS 1#x] + +- Range (0, 10, step=1, splits=None) + + +-- !query +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(max(id) > 0)\"", + "expressionList" : "max(id)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 42, + "fragment" : "SELECT 1 FROM range(10) HAVING MAX(id) > 0" + } ] +} + + +-- !query +SELECT id FROM range(10) HAVING id > 0 +-- !query analysis +Filter (id#xL > cast(0 as bigint)) ++- Project [id#xL] + +- Range (0, 10, step=1, splits=None) + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false +-- !query analysis +SetCommand (spark.sql.legacy.parser.havingWithoutGroupByAsWhere,Some(false)) + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v) +-- !query analysis +CreateViewCommand `test_agg`, SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v), false, true, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0 +-- !query analysis +Aggregate [every(v#x) AS every(v)#x, some(v#x) AS some(v)#x, any(v#x) AS any(v)#x, bool_and(v#x) AS bool_and(v)#x, bool_or(v#x) AS bool_or(v)#x] ++- Filter (1 = 0) + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4 +-- !query analysis +Aggregate [every(v#x) AS every(v)#x, some(v#x) AS some(v)#x, any(v#x) AS any(v)#x, bool_and(v#x) AS bool_and(v)#x, bool_or(v#x) AS bool_or(v)#x] ++- Filter (k#x = 4) + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5 +-- !query analysis +Aggregate [every(v#x) AS every(v)#x, some(v#x) AS some(v)#x, any(v#x) AS any(v)#x, bool_and(v#x) AS bool_and(v)#x, bool_or(v#x) AS bool_or(v)#x] ++- Filter (k#x = 5) + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, every(v#x) AS every(v)#x, some(v#x) AS some(v)#x, any(v#x) AS any(v)#x, bool_and(v#x) AS bool_and(v)#x, bool_or(v#x) AS bool_or(v)#x] ++- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false +-- !query analysis +Filter (every(v)#x = false) ++- Aggregate [k#x], [k#x, every(v#x) AS every(v)#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL +-- !query analysis +Filter isnull(every(v)#x) ++- Aggregate [k#x], [k#x, every(v#x) AS every(v)#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, every(v#x) AS every#x] ++- Filter ((k#x = 2) AND v#x IN (list#x [])) + : +- Aggregate [any(v#x) AS any(v)#x] + : +- Filter (k#x = 1) + : +- SubqueryAlias test_agg + : +- View (`test_agg`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias test_agg + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, every(v#x) AS every#x] ++- Filter ((k#x = 2) AND v#x IN (list#x [])) + : +- Aggregate [every(v#x) AS every(v)#x] + : +- Filter (k#x = 1) + : +- SubqueryAlias test_agg + : +- View (`test_agg`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias test_agg + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT every(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "every(1)" + } ] +} + + +-- !query +SELECT some(1S) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"some(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "some(1S)" + } ] +} + + +-- !query +SELECT any(1L) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"any(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "any(1L)" + } ] +} + + +-- !query +SELECT every("true") +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"true\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "every(\"true\")" + } ] +} + + +-- !query +SELECT bool_and(1.0) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"bool_and(1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "bool_and(1.0)" + } ] +} + + +-- !query +SELECT bool_or(1.0D) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"bool_or(1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "bool_or(1.0D)" + } ] +} + + +-- !query +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query analysis +Project [k#x, v#x, every(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [k#x, v#x, every(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, every(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [every(v#x) windowspecdefinition(k#x, v#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS every(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [k#x], [v#x ASC NULLS FIRST] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query analysis +Project [k#x, v#x, some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [k#x, v#x, some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [some(v#x) windowspecdefinition(k#x, v#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [k#x], [v#x ASC NULLS FIRST] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query analysis +Project [k#x, v#x, any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [k#x, v#x, any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [any(v#x) windowspecdefinition(k#x, v#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [k#x], [v#x ASC NULLS FIRST] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, v, bool_and(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query analysis +Project [k#x, v#x, bool_and(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [k#x, v#x, bool_and(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, bool_and(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [bool_and(v#x) windowspecdefinition(k#x, v#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS bool_and(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [k#x], [v#x ASC NULLS FIRST] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, v, bool_or(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query analysis +Project [k#x, v#x, bool_or(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [k#x, v#x, bool_or(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, bool_or(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [bool_or(v#x) windowspecdefinition(k#x, v#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS bool_or(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [k#x], [v#x ASC NULLS FIRST] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT count(*) FROM test_agg HAVING count(*) > 1L +-- !query analysis +Filter (count(1)#xL > 1) ++- Aggregate [count(1) AS count(1)#xL] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true +-- !query analysis +Filter (max(v)#x = true) ++- Aggregate [k#x], [k#x, max(v#x) AS max(v)#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L +-- !query analysis +Project [cnt#xL] ++- Filter (cnt#xL > 1) + +- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [count(1) AS cnt#xL] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT count(*) FROM test_agg WHERE count(*) > 1L +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(count(1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "SELECT count(*) FROM test_agg WHERE count(*) > 1L" + } ] +} + + +-- !query +SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"((count(1) + 1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L" + } ] +} + + +-- !query +SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(((k = 1) OR (k = 2)) OR (((count(1) + 1) > 1) OR (max(k) > 1)))\"", + "expressionList" : "count(1), max(test_agg.k)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1" + } ] +} + + +-- !query +SELECT AVG(DISTINCT decimal_col), SUM(DISTINCT decimal_col) FROM VALUES (CAST(1 AS DECIMAL(9, 0))) t(decimal_col) +-- !query analysis +Aggregate [avg(distinct decimal_col#x) AS avg(DISTINCT decimal_col)#x, sum(distinct decimal_col#x) AS sum(DISTINCT decimal_col)#x] ++- SubqueryAlias t + +- LocalRelation [decimal_col#x] + + +-- !query +SELECT not(a IS NULL), count(*) AS c +FROM testData +GROUP BY a IS NULL +-- !query analysis +Aggregate [isnull(a#x)], [NOT isnull(a#x) AS (NOT (a IS NULL))#x, count(1) AS c#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT if(not(a IS NULL), rand(0), 1), count(*) AS c +FROM testData +GROUP BY a IS NULL +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT + histogram_numeric(col, 2) as histogram_2, + histogram_numeric(col, 3) as histogram_3, + histogram_numeric(col, 5) as histogram_5, + histogram_numeric(col, 10) as histogram_10 +FROM VALUES + (1), (2), (3), (4), (5), (6), (7), (8), (9), (10), + (11), (12), (13), (14), (15), (16), (17), (18), (19), (20), + (21), (22), (23), (24), (25), (26), (27), (28), (29), (30), + (31), (32), (33), (34), (35), (3), (37), (38), (39), (40), + (41), (42), (43), (44), (45), (46), (47), (48), (49), (50) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#x, 2, 0, 0) AS histogram_2#x, histogram_numeric(col#x, 3, 0, 0) AS histogram_3#x, histogram_numeric(col#x, 5, 0, 0) AS histogram_5#x, histogram_numeric(col#x, 10, 0, 0) AS histogram_10#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1), (2), (3) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#x, 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1L), (2L), (3L) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#xL, 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#xL] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1F), (2F), (3F) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#x, 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1D), (2D), (3D) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#x, 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1S), (2S), (3S) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#x, 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BYTE)), (CAST(2 AS BYTE)), (CAST(3 AS BYTE)) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#x, 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS TINYINT)), (CAST(2 AS TINYINT)), (CAST(3 AS TINYINT)) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#x, 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS SMALLINT)), (CAST(2 AS SMALLINT)), (CAST(3 AS SMALLINT)) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#x, 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BIGINT)), (CAST(2 AS BIGINT)), (CAST(3 AS BIGINT)) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#xL, 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#xL] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (TIMESTAMP '2017-03-01 00:00:00'), + (TIMESTAMP '2017-04-01 00:00:00'), (TIMESTAMP '2017-05-01 00:00:00') AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#x, 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '100-00' YEAR TO MONTH), + (INTERVAL '110-00' YEAR TO MONTH), (INTERVAL '120-00' YEAR TO MONTH) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#x, 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '12 20:4:0' DAY TO SECOND), + (INTERVAL '12 21:4:0' DAY TO SECOND), (INTERVAL '12 22:4:0' DAY TO SECOND) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#x, 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (NULL), (NULL), (NULL) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(cast(col#x as double), 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#x, 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS INT)), (CAST(NULL AS INT)), (CAST(NULL AS INT)) AS tab(col) +-- !query analysis +Aggregate [histogram_numeric(col#x, 3, 0, 0) AS histogram_numeric(col, 3)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT + collect_list(col), + array_agg(col) +FROM VALUES + (1), (2), (1) AS tab(col) +-- !query analysis +Aggregate [collect_list(col#x, 0, 0) AS collect_list(col)#x, collect_list(col#x, 0, 0) AS collect_list(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT + a, + collect_list(b), + array_agg(b) +FROM VALUES + (1,4),(2,3),(1,4),(2,4) AS v(a,b) +GROUP BY a +-- !query analysis +Aggregate [a#x], [a#x, collect_list(b#x, 0, 0) AS collect_list(b)#x, collect_list(b#x, 0, 0) AS collect_list(b)#x] ++- SubqueryAlias v + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT mode(a), mode(b) FROM testData +-- !query analysis +Aggregate [mode(a#x, 0, 0) AS mode(a)#x, mode(b#x, 0, 0) AS mode(b)#x] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a, mode(b) FROM testData GROUP BY a ORDER BY a +-- !query analysis +Sort [a#x ASC NULLS FIRST], true ++- Aggregate [a#x], [a#x, mode(b#x, 0, 0) AS mode(b)#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out new file mode 100644 index 0000000000000..1a2cefd16498a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out @@ -0,0 +1,242 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW grouping AS SELECT * FROM VALUES + ("1", "2", "3", 1), + ("4", "5", "6", 1), + ("7", "8", "9", 1) + as grouping(a, b, c, d) +-- !query analysis +CreateViewCommand `grouping`, SELECT * FROM VALUES + ("1", "2", "3", 1), + ("4", "5", "6", 1), + ("7", "8", "9", 1) + as grouping(a, b, c, d), false, false, LocalTempView, true + +- Project [a#x, b#x, c#x, d#x] + +- SubqueryAlias grouping + +- LocalRelation [a#x, b#x, c#x, d#x] + + +-- !query +SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS (()) +-- !query analysis +Aggregate [a#x, b#x, c#x, spark_grouping_id#xL], [a#x, b#x, c#x, count(d#x) AS count(d)#xL] ++- Expand [[a#x, b#x, c#x, d#x, null, null, null, 7]], [a#x, b#x, c#x, d#x, a#x, b#x, c#x, spark_grouping_id#xL] + +- Project [a#x, b#x, c#x, d#x, a#x AS a#x, b#x AS b#x, c#x AS c#x] + +- SubqueryAlias grouping + +- View (`grouping`, [a#x,b#x,c#x,d#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x, cast(c#x as string) AS c#x, cast(d#x as int) AS d#x] + +- Project [a#x, b#x, c#x, d#x] + +- SubqueryAlias grouping + +- LocalRelation [a#x, b#x, c#x, d#x] + + +-- !query +SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((a)) +-- !query analysis +Aggregate [a#x, b#x, c#x, spark_grouping_id#xL], [a#x, b#x, c#x, count(d#x) AS count(d)#xL] ++- Expand [[a#x, b#x, c#x, d#x, a#x, null, null, 3]], [a#x, b#x, c#x, d#x, a#x, b#x, c#x, spark_grouping_id#xL] + +- Project [a#x, b#x, c#x, d#x, a#x AS a#x, b#x AS b#x, c#x AS c#x] + +- SubqueryAlias grouping + +- View (`grouping`, [a#x,b#x,c#x,d#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x, cast(c#x as string) AS c#x, cast(d#x as int) AS d#x] + +- Project [a#x, b#x, c#x, d#x] + +- SubqueryAlias grouping + +- LocalRelation [a#x, b#x, c#x, d#x] + + +-- !query +SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((c)) +-- !query analysis +Aggregate [a#x, b#x, c#x, spark_grouping_id#xL], [a#x, b#x, c#x, count(d#x) AS count(d)#xL] ++- Expand [[a#x, b#x, c#x, d#x, null, null, c#x, 6]], [a#x, b#x, c#x, d#x, a#x, b#x, c#x, spark_grouping_id#xL] + +- Project [a#x, b#x, c#x, d#x, a#x AS a#x, b#x AS b#x, c#x AS c#x] + +- SubqueryAlias grouping + +- View (`grouping`, [a#x,b#x,c#x,d#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x, cast(c#x as string) AS c#x, cast(d#x as int) AS d#x] + +- Project [a#x, b#x, c#x, d#x] + +- SubqueryAlias grouping + +- LocalRelation [a#x, b#x, c#x, d#x] + + +-- !query +SELECT c1, sum(c2) FROM (VALUES ('x', 10, 0), ('y', 20, 0)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1) +-- !query analysis +Aggregate [c1#x, spark_grouping_id#xL], [c1#x, sum(c2#x) AS sum(c2)#xL] ++- Expand [[c1#x, c2#x, c3#x, c1#x, 0]], [c1#x, c2#x, c3#x, c1#x, spark_grouping_id#xL] + +- Project [c1#x, c2#x, c3#x, c1#x AS c1#x] + +- SubqueryAlias t + +- Project [col1#x AS c1#x, col2#x AS c2#x, col3#x AS c3#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT c1, sum(c2), grouping(c1) FROM (VALUES ('x', 10, 0), ('y', 20, 0)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1) +-- !query analysis +Aggregate [c1#x, spark_grouping_id#xL], [c1#x, sum(c2#x) AS sum(c2)#xL, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(c1)#x] ++- Expand [[c1#x, c2#x, c3#x, c1#x, 0]], [c1#x, c2#x, c3#x, c1#x, spark_grouping_id#xL] + +- Project [c1#x, c2#x, c3#x, c1#x AS c1#x] + +- SubqueryAlias t + +- Project [col1#x AS c1#x, col2#x AS c2#x, col3#x AS c3#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT c1, c2, Sum(c3), grouping__id +FROM (VALUES ('x', 'a', 10), ('y', 'b', 20) ) AS t (c1, c2, c3) +GROUP BY GROUPING SETS ( ( c1 ), ( c2 ) ) +HAVING GROUPING__ID > 1 +-- !query analysis +Filter (grouping__id#xL > cast(1 as bigint)) ++- Aggregate [c1#x, c2#x, spark_grouping_id#xL], [c1#x, c2#x, sum(c3#x) AS sum(c3)#xL, spark_grouping_id#xL AS grouping__id#xL] + +- Expand [[c1#x, c2#x, c3#x, c1#x, null, 1], [c1#x, c2#x, c3#x, null, c2#x, 2]], [c1#x, c2#x, c3#x, c1#x, c2#x, spark_grouping_id#xL] + +- Project [c1#x, c2#x, c3#x, c1#x AS c1#x, c2#x AS c2#x] + +- SubqueryAlias t + +- Project [col1#x AS c1#x, col2#x AS c2#x, col3#x AS c3#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT grouping(c1) FROM (VALUES ('x', 'a', 10), ('y', 'b', 20)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1,c2) +-- !query analysis +Aggregate [c1#x, c2#x, spark_grouping_id#xL], [cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(c1)#x] ++- Expand [[c1#x, c2#x, c3#x, c1#x, null, 1], [c1#x, c2#x, c3#x, null, c2#x, 2]], [c1#x, c2#x, c3#x, c1#x, c2#x, spark_grouping_id#xL] + +- Project [c1#x, c2#x, c3#x, c1#x AS c1#x, c2#x AS c2#x] + +- SubqueryAlias t + +- Project [col1#x AS c1#x, col2#x AS c2#x, col3#x AS c3#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT -c1 AS c1 FROM (values (1,2), (3,2)) t(c1, c2) GROUP BY GROUPING SETS ((c1), (c1, c2)) +-- !query analysis +Aggregate [c1#x, c2#x, spark_grouping_id#xL], [-c1#x AS c1#x] ++- Expand [[c1#x, c2#x, c1#x, null, 1], [c1#x, c2#x, c1#x, c2#x, 0]], [c1#x, c2#x, c1#x, c2#x, spark_grouping_id#xL] + +- Project [c1#x, c2#x, c1#x AS c1#x, c2#x AS c2#x] + +- SubqueryAlias t + +- Project [col1#x AS c1#x, col2#x AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT a + b, b, sum(c) FROM (VALUES (1,1,1),(2,2,2)) AS t(a,b,c) GROUP BY GROUPING SETS ( (a + b), (b)) +-- !query analysis +Aggregate [(a#x + b#x)#x, b#x, spark_grouping_id#xL], [(a#x + b#x)#x AS (a + b)#x, b#x, sum(c#x) AS sum(c)#xL] ++- Expand [[a#x, b#x, c#x, (a#x + b#x)#x, null, 1], [a#x, b#x, c#x, null, b#x, 2]], [a#x, b#x, c#x, (a#x + b#x)#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, c#x, (a#x + b#x) AS (a#x + b#x)#x, b#x AS b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT a + b, b, sum(c) FROM (VALUES (1,1,1),(2,2,2)) AS t(a,b,c) GROUP BY GROUPING SETS ( (a + b), (b + a), (b)) +-- !query analysis +Aggregate [(a#x + b#x)#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [(a#x + b#x)#x AS (a + b)#x, b#x, sum(c#x) AS sum(c)#xL] ++- Expand [[a#x, b#x, c#x, (a#x + b#x)#x, null, 1, 0], [a#x, b#x, c#x, (a#x + b#x)#x, null, 1, 1], [a#x, b#x, c#x, null, b#x, 2, 2]], [a#x, b#x, c#x, (a#x + b#x)#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, c#x, (a#x + b#x) AS (a#x + b#x)#x, b#x AS b#x] + +- SubqueryAlias t + +- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT c1 AS col1, c2 AS col2 +FROM (VALUES (1, 2), (3, 2)) t(c1, c2) +GROUP BY GROUPING SETS ( ( c1 ), ( c1, c2 ) ) +HAVING col2 IS NOT NULL +ORDER BY -col1 +-- !query analysis +Sort [-col1#x ASC NULLS FIRST], true ++- Filter isnotnull(col2#x) + +- Aggregate [c1#x, c2#x, spark_grouping_id#xL], [c1#x AS col1#x, c2#x AS col2#x] + +- Expand [[c1#x, c2#x, c1#x, null, 1], [c1#x, c2#x, c1#x, c2#x, 0]], [c1#x, c2#x, c1#x, c2#x, spark_grouping_id#xL] + +- Project [c1#x, c2#x, c1#x AS c1#x, c2#x AS c2#x] + +- SubqueryAlias t + +- Project [col1#x AS c1#x, col2#x AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT a, b, c, count(d) FROM grouping GROUP BY WITH ROLLUP +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'ROLLUP'", + "hint" : ": extra input 'ROLLUP'" + } +} + + +-- !query +SELECT a, b, c, count(d) FROM grouping GROUP BY WITH CUBE +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'CUBE'", + "hint" : ": extra input 'CUBE'" + } +} + + +-- !query +SELECT c1 FROM (values (1,2), (3,2)) t(c1, c2) GROUP BY GROUPING SETS (()) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"c1\"", + "expressionAnyValue" : "\"any_value(c1)\"" + } +} + + +-- !query +SELECT k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) +-- !query analysis +Aggregate [k1#x, k2#x, spark_grouping_id#xL, _gen_grouping_pos#x], [k1#x, k2#x, avg(v#x) AS avg(v)#x] ++- Expand [[k1#x, k2#x, v#x, k1#x, null, 1, 0], [k1#x, k2#x, v#x, k1#x, k2#x, 0, 1], [k1#x, k2#x, v#x, k1#x, k2#x, 0, 2]], [k1#x, k2#x, v#x, k1#x, k2#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [k1#x, k2#x, v#x, k1#x AS k1#x, k2#x AS k2#x] + +- SubqueryAlias t + +- Project [col1#x AS k1#x, col2#x AS k2#x, col3#x AS v#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT grouping__id, k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) +-- !query analysis +Aggregate [k1#x, k2#x, spark_grouping_id#xL, _gen_grouping_pos#x], [spark_grouping_id#xL AS grouping__id#xL, k1#x, k2#x, avg(v#x) AS avg(v)#x] ++- Expand [[k1#x, k2#x, v#x, k1#x, null, 1, 0], [k1#x, k2#x, v#x, k1#x, k2#x, 0, 1], [k1#x, k2#x, v#x, k1#x, k2#x, 0, 2]], [k1#x, k2#x, v#x, k1#x, k2#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [k1#x, k2#x, v#x, k1#x AS k1#x, k2#x AS k2#x] + +- SubqueryAlias t + +- Project [col1#x AS k1#x, col2#x AS k2#x, col3#x AS v#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT grouping(k1), k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) +-- !query analysis +Aggregate [k1#x, k2#x, spark_grouping_id#xL, _gen_grouping_pos#x], [cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(k1)#x, k1#x, k2#x, avg(v#x) AS avg(v)#x] ++- Expand [[k1#x, k2#x, v#x, k1#x, null, 1, 0], [k1#x, k2#x, v#x, k1#x, k2#x, 0, 1], [k1#x, k2#x, v#x, k1#x, k2#x, 0, 2]], [k1#x, k2#x, v#x, k1#x, k2#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [k1#x, k2#x, v#x, k1#x AS k1#x, k2#x AS k2#x] + +- SubqueryAlias t + +- Project [col1#x AS k1#x, col2#x AS k2#x, col3#x AS v#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT grouping_id(k1, k2), avg(v) from (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY k1, k2 GROUPING SETS ((k2, k1), k1) +-- !query analysis +Aggregate [k1#x, k2#x, spark_grouping_id#xL], [spark_grouping_id#xL AS grouping_id(k1, k2)#xL, avg(v#x) AS avg(v)#x] ++- Expand [[k1#x, k2#x, v#x, k1#x, k2#x, 0], [k1#x, k2#x, v#x, k1#x, null, 1]], [k1#x, k2#x, v#x, k1#x, k2#x, spark_grouping_id#xL] + +- Project [k1#x, k2#x, v#x, k1#x AS k1#x, k2#x AS k2#x] + +- SubqueryAlias t + +- Project [col1#x AS k1#x, col2#x AS k2#x, col3#x AS v#x] + +- LocalRelation [col1#x, col2#x, col3#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/having.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/having.sql.out new file mode 100644 index 0000000000000..ec67de27390b5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/having.sql.out @@ -0,0 +1,181 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view hav as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v) +-- !query analysis +CreateViewCommand `hav`, select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v), false, false, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias hav + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2 +-- !query analysis +Filter (sum(v)#xL > cast(2 as bigint)) ++- Aggregate [k#x], [k#x, sum(v#x) AS sum(v)#xL] + +- SubqueryAlias hav + +- View (`hav`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias hav + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT count(k) FROM hav GROUP BY v + 1 HAVING v + 1 = 2 +-- !query analysis +Project [count(k)#xL] ++- Filter ((v#x + 1)#x = 2) + +- Aggregate [(v#x + 1)], [count(k#x) AS count(k)#xL, (v#x + 1) AS (v#x + 1)#x] + +- SubqueryAlias hav + +- View (`hav`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias hav + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT count(k) FROM hav GROUP BY v HAVING v = array(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"ARRAY\"", + "sqlExpr" : "\"(v = array(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 55, + "fragment" : "v = array(1)" + } ] +} + + +-- !query +SELECT MIN(t.v) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(COUNT(1) > 0) +-- !query analysis +Project [min(v)#x] ++- Filter (count(1)#xL > cast(0 as bigint)) + +- Aggregate [min(v#x) AS min(v)#x, count(1) AS count(1)#xL] + +- SubqueryAlias t + +- Project [k#x, v#x] + +- Filter (v#x > 0) + +- SubqueryAlias hav + +- View (`hav`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias hav + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT a + b FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > 1 +-- !query analysis +Filter ((a + b)#xL > cast(1 as bigint)) ++- Aggregate [(a#xL + cast(b#x as bigint))], [(a#xL + cast(b#x as bigint)) AS (a + b)#xL] + +- SubqueryAlias T + +- LocalRelation [a#xL, b#x] + + +-- !query +SELECT SUM(a) AS b, CAST('2020-01-01' AS DATE) AS fake FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY b HAVING b > 10 +-- !query analysis +Project [b#xL, fake#x] ++- Filter (b#x > 10) + +- Aggregate [b#x], [sum(a#x) AS b#xL, cast(2020-01-01 as date) AS fake#x, b#x] + +- SubqueryAlias T + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT SUM(a) AS b FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY GROUPING SETS ((b), (a, b)) HAVING b > 10 +-- !query analysis +Project [b#xL] ++- Filter (b#x > 10) + +- Aggregate [b#x, a#x, spark_grouping_id#xL], [sum(a#x) AS b#xL, b#x] + +- Expand [[a#x, b#x, b#x, null, 1], [a#x, b#x, b#x, a#x, 0]], [a#x, b#x, b#x, a#x, spark_grouping_id#xL] + +- Project [a#x, b#x, b#x AS b#x, a#x AS a#x] + +- SubqueryAlias T + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT SUM(a) AS b FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY CUBE(a, b) HAVING b > 10 +-- !query analysis +Project [b#xL] ++- Filter (b#x > 10) + +- Aggregate [a#x, b#x, spark_grouping_id#xL], [sum(a#x) AS b#xL, b#x] + +- Expand [[a#x, b#x, a#x, b#x, 0], [a#x, b#x, a#x, null, 1], [a#x, b#x, null, b#x, 2], [a#x, b#x, null, null, 3]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias T + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT SUM(a) AS b FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY ROLLUP(a, b) HAVING b > 10 +-- !query analysis +Project [b#xL] ++- Filter (b#x > 10) + +- Aggregate [a#x, b#x, spark_grouping_id#xL], [sum(a#x) AS b#xL, b#x] + +- Expand [[a#x, b#x, a#x, b#x, 0], [a#x, b#x, a#x, null, 1], [a#x, b#x, null, null, 3]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias T + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT c1 FROM VALUES (1, 2) as t(c1, c2) GROUP BY GROUPING SETS(t.c1) HAVING t.c1 = 1 +-- !query analysis +Filter (c1#x = 1) ++- Aggregate [c1#x, spark_grouping_id#xL], [c1#x] + +- Expand [[c1#x, c2#x, c1#x, 0]], [c1#x, c2#x, c1#x, spark_grouping_id#xL] + +- Project [c1#x, c2#x, c1#x AS c1#x] + +- SubqueryAlias t + +- LocalRelation [c1#x, c2#x] + + +-- !query +SELECT c1 FROM VALUES (1, 2) as t(c1, c2) GROUP BY CUBE(t.c1) HAVING t.c1 = 1 +-- !query analysis +Filter (c1#x = 1) ++- Aggregate [c1#x, spark_grouping_id#xL], [c1#x] + +- Expand [[c1#x, c2#x, c1#x, 0], [c1#x, c2#x, null, 1]], [c1#x, c2#x, c1#x, spark_grouping_id#xL] + +- Project [c1#x, c2#x, c1#x AS c1#x] + +- SubqueryAlias t + +- LocalRelation [c1#x, c2#x] + + +-- !query +SELECT c1 FROM VALUES (1, 2) as t(c1, c2) GROUP BY ROLLUP(t.c1) HAVING t.c1 = 1 +-- !query analysis +Filter (c1#x = 1) ++- Aggregate [c1#x, spark_grouping_id#xL], [c1#x] + +- Expand [[c1#x, c2#x, c1#x, 0], [c1#x, c2#x, null, 1]], [c1#x, c2#x, c1#x, spark_grouping_id#xL] + +- Project [c1#x, c2#x, c1#x AS c1#x] + +- SubqueryAlias t + +- LocalRelation [c1#x, c2#x] + + +-- !query +SELECT c1 FROM VALUES (1, 2) as t(c1, c2) GROUP BY t.c1 HAVING t.c1 = 1 +-- !query analysis +Filter (c1#x = 1) ++- Aggregate [c1#x], [c1#x] + +- SubqueryAlias t + +- LocalRelation [c1#x, c2#x] 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 new file mode 100644 index 0000000000000..de90751777cad --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/higher-order-functions.sql.out @@ -0,0 +1,351 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create or replace temporary view nested as values + (1, array(32, 97), array(array(12, 99), array(123, 42), array(1))), + (2, array(77, -76), array(array(6, 96, 65), array(-1, -2))), + (3, array(12), array(array(17))) + as t(x, ys, zs) +-- !query analysis +CreateViewCommand `nested`, values + (1, array(32, 97), array(array(12, 99), array(123, 42), array(1))), + (2, array(77, -76), array(array(6, 96, 65), array(-1, -2))), + (3, array(12), array(array(17))) + as t(x, ys, zs), false, true, LocalTempView, true + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select upper(x -> x) as v +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2306", + "messageParameters" : { + "class" : "org.apache.spark.sql.catalyst.expressions.Upper" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "upper(x -> x)" + } ] +} + + +-- !query +select transform(zs, z -> z) as v from nested +-- !query analysis +Project [transform(zs#x, lambdafunction(lambda z#x, 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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select transform(ys, y -> y * y) as v from nested +-- !query analysis +Project [transform(ys#x, lambdafunction((lambda y#x * lambda y#x), lambda y#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select transform(ys, (y, i) -> y + i) as v from nested +-- !query analysis +Project [transform(ys#x, lambdafunction((lambda y#x + lambda i#x), lambda y#x, lambda i#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select transform(zs, z -> concat(ys, z)) as v from nested +-- !query analysis +Project [transform(zs#x, lambdafunction(concat(ys#x, lambda z#x), 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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select transform(ys, 0) as v from nested +-- !query analysis +Project [transform(ys#x, lambdafunction(0, lambda col0#x, true)) 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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select transform(cast(null as array), x -> x + 1) as v +-- !query analysis +Project [transform(cast(null as array), lambdafunction((lambda x#x + 1), lambda x#x, false)) AS v#x] ++- OneRowRelation + + +-- !query +select filter(ys, y -> y > 30) as v from nested +-- !query analysis +Project [filter(ys#x, lambdafunction((lambda y#x > 30), lambda y#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select filter(cast(null as array), y -> true) as v +-- !query analysis +Project [filter(cast(null as array), lambdafunction(true, lambda y#x, false)) AS v#x] ++- OneRowRelation + + +-- !query +select transform(zs, z -> filter(z, zz -> zz > 50)) as v from nested +-- !query analysis +Project [transform(zs#x, lambdafunction(filter(lambda z#x, lambdafunction((lambda zz#x > 50), lambda zz#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select aggregate(ys, 0, (y, a) -> y + a + x) as v from nested +-- !query analysis +Project [aggregate(ys#x, 0, lambdafunction(((lambda y#x + lambda a#x) + x#x), lambda y#x, lambda a#x, false), lambdafunction(lambda id#x, lambda id#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select aggregate(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested +-- !query analysis +Project [aggregate(ys#x, named_struct(sum, 0, n, 0), lambdafunction(named_struct(col1, (lambda acc#x.sum + lambda x#x), col2, (lambda acc#x.n + 1)), lambda acc#x, lambda x#x, false), lambdafunction((cast(lambda acc#x.sum as double) / cast(lambda acc#x.n as double)), lambda acc#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !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] ++- 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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select aggregate(cast(null as array), 0, (a, y) -> a + y + 1, a -> a + 2) as v +-- !query analysis +Project [aggregate(cast(null as array), 0, lambdafunction(((lambda a#x + lambda y#x) + 1), lambda a#x, lambda y#x, false), lambdafunction((lambda a#x + 2), lambda a#x, false)) AS v#x] ++- OneRowRelation + + +-- !query +select reduce(ys, 0, (y, a) -> y + a + x) as v from nested +-- !query analysis +Project [reduce(ys#x, 0, lambdafunction(((lambda y#x + lambda a#x) + x#x), lambda y#x, lambda a#x, false), lambdafunction(lambda id#x, lambda id#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select reduce(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested +-- !query analysis +Project [reduce(ys#x, named_struct(sum, 0, n, 0), lambdafunction(named_struct(col1, (lambda acc#x.sum + lambda x#x), col2, (lambda acc#x.n + 1)), lambda acc#x, lambda x#x, false), lambdafunction((cast(lambda acc#x.sum as double) / cast(lambda acc#x.n as double)), lambda acc#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !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] ++- 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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select reduce(cast(null as array), 0, (a, y) -> a + y + 1, a -> a + 2) as v +-- !query analysis +Project [reduce(cast(null as array), 0, lambdafunction(((lambda a#x + lambda y#x) + 1), lambda a#x, lambda y#x, false), lambdafunction((lambda a#x + 2), lambda a#x, false)) AS v#x] ++- OneRowRelation + + +-- !query +select exists(ys, y -> y > 30) as v from nested +-- !query analysis +Project [exists(ys#x, lambdafunction((lambda y#x > 30), lambda y#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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select exists(cast(null as array), y -> y > 30) as v +-- !query analysis +Project [exists(cast(null as array), lambdafunction((lambda y#x > 30), lambda y#x, false)) AS v#x] ++- OneRowRelation + + +-- !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] ++- 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] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x, zs#x] + + +-- !query +select zip_with(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) -> concat(x, y)) as v +-- !query analysis +Project [zip_with(array(a, b, c), array(d, e, f), lambdafunction(concat(lambda x#x, lambda y#x), lambda x#x, lambda y#x, false)) AS v#x] ++- OneRowRelation + + +-- !query +select zip_with(array('a'), array('d', null, 'f'), (x, y) -> coalesce(x, y)) as v +-- !query analysis +Project [zip_with(array(a), array(d, cast(null as string), f), lambdafunction(coalesce(lambda x#x, lambda y#x), lambda x#x, lambda y#x, false)) AS v#x] ++- OneRowRelation + + +-- !query +create or replace temporary view nested as values + (1, map(1, 1, 2, 2, 3, 3)), + (2, map(4, 4, 5, 5, 6, 6)) + as t(x, ys) +-- !query analysis +CreateViewCommand `nested`, values + (1, map(1, 1, 2, 2, 3, 3)), + (2, map(4, 4, 5, 5, 6, 6)) + as t(x, ys), false, true, LocalTempView, true + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x] + + +-- !query +select transform_keys(ys, (k, v) -> k) as v from nested +-- !query analysis +Project [transform_keys(ys#x, lambdafunction(lambda k#x, lambda k#x, lambda v#x, false)) AS v#x] ++- SubqueryAlias nested + +- View (`nested`, [x#x,ys#x]) + +- Project [cast(x#x as int) AS x#x, cast(ys#x as map) AS ys#x] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x] + + +-- !query +select transform_keys(ys, (k, v) -> k + 1) as v from nested +-- !query analysis +Project [transform_keys(ys#x, lambdafunction((lambda k#x + 1), lambda k#x, lambda v#x, false)) AS v#x] ++- SubqueryAlias nested + +- View (`nested`, [x#x,ys#x]) + +- Project [cast(x#x as int) AS x#x, cast(ys#x as map) AS ys#x] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x] + + +-- !query +select transform_keys(ys, (k, v) -> k + v) as v from nested +-- !query analysis +Project [transform_keys(ys#x, lambdafunction((lambda k#x + lambda v#x), lambda k#x, lambda v#x, false)) AS v#x] ++- SubqueryAlias nested + +- View (`nested`, [x#x,ys#x]) + +- Project [cast(x#x as int) AS x#x, cast(ys#x as map) AS ys#x] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x] + + +-- !query +select transform_values(ys, (k, v) -> v) as v from nested +-- !query analysis +Project [transform_values(ys#x, lambdafunction(lambda v#x, lambda k#x, lambda v#x, false)) AS v#x] ++- SubqueryAlias nested + +- View (`nested`, [x#x,ys#x]) + +- Project [cast(x#x as int) AS x#x, cast(ys#x as map) AS ys#x] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x] + + +-- !query +select transform_values(ys, (k, v) -> v + 1) as v from nested +-- !query analysis +Project [transform_values(ys#x, lambdafunction((lambda v#x + 1), lambda k#x, lambda v#x, false)) AS v#x] ++- SubqueryAlias nested + +- View (`nested`, [x#x,ys#x]) + +- Project [cast(x#x as int) AS x#x, cast(ys#x as map) AS ys#x] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x] + + +-- !query +select transform_values(ys, (k, v) -> k + v) as v from nested +-- !query analysis +Project [transform_values(ys#x, lambdafunction((lambda k#x + lambda v#x), lambda k#x, lambda v#x, false)) AS v#x] ++- SubqueryAlias nested + +- View (`nested`, [x#x,ys#x]) + +- Project [cast(x#x as int) AS x#x, cast(ys#x as map) AS ys#x] + +- SubqueryAlias t + +- LocalRelation [x#x, ys#x] + + +-- !query +select transform(ys, all -> all * all) as v from values (array(32, 97)) as t(ys) +-- !query analysis +Project [transform(ys#x, lambdafunction((lambda all#x * lambda all#x), lambda all#x, false)) AS v#x] ++- SubqueryAlias t + +- LocalRelation [ys#x] + + +-- !query +select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t(ys) +-- !query analysis +Project [transform(ys#x, lambdafunction((lambda all#x + lambda i#x), lambda all#x, lambda i#x, false)) AS v#x] ++- SubqueryAlias t + +- LocalRelation [ys#x] + + +-- !query +select aggregate(split('abcdefgh',''), array(array('')), (acc, x) -> array(array(x))) +-- !query analysis +Project [aggregate(split(abcdefgh, , -1), array(array()), lambdafunction(array(array(lambda x#x)), lambda acc#x, lambda x#x, false), lambdafunction(lambda id#x, lambda id#x, false)) AS aggregate(split(abcdefgh, , -1), array(array()), lambdafunction(array(array(namedlambdavariable())), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable()))#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ilike-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ilike-all.sql.out new file mode 100644 index 0000000000000..aeb9fcdcb9890 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ilike-all.sql.out @@ -0,0 +1,213 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW ilike_all_table AS SELECT * FROM (VALUES + ('gOOgle', '%oo%'), + ('facebook', '%OO%'), + ('liNkedin', '%In')) + as t1(company, pat) +-- !query analysis +CreateViewCommand `ilike_all_table`, SELECT * FROM (VALUES + ('gOOgle', '%oo%'), + ('facebook', '%OO%'), + ('liNkedin', '%In')) + as t1(company, pat), false, true, LocalTempView, true + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_all_table WHERE company ILIKE ALL ('%oO%', '%Go%') +-- !query analysis +Project [company#x] ++- Filter likeall(lower(company#x), %oo%, %go%) + +- SubqueryAlias ilike_all_table + +- View (`ilike_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_all_table WHERE company ILIKE ALL ('microsoft', '%yoo%') +-- !query analysis +Project [company#x] ++- Filter likeall(lower(company#x), microsoft, %yoo%) + +- SubqueryAlias ilike_all_table + +- View (`ilike_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT + company, + CASE + WHEN company ILIKE ALL ('%oo%', '%GO%') THEN 'Y' + ELSE 'N' + END AS is_available, + CASE + WHEN company ILIKE ALL ('%OO%', 'go%') OR company ILIKE ALL ('%IN', 'ms%') THEN 'Y' + ELSE 'N' + END AS mix +FROM ilike_all_table +-- !query analysis +Project [company#x, CASE WHEN likeall(lower(company#x), %oo%, %go%) THEN Y ELSE N END AS is_available#x, CASE WHEN (likeall(lower(company#x), %oo%, go%) OR likeall(lower(company#x), %in, ms%)) THEN Y ELSE N END AS mix#x] ++- SubqueryAlias ilike_all_table + +- View (`ilike_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_all_table WHERE company ILIKE ALL ('%oo%', pat) +-- !query analysis +Project [company#x] ++- Filter (ilike(company#x, %oo%, \) AND ilike(company#x, pat#x, \)) + +- SubqueryAlias ilike_all_table + +- View (`ilike_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_all_table WHERE company NOT ILIKE ALL ('%oo%', '%In', 'Fa%') +-- !query analysis +Project [company#x] ++- Filter notlikeall(lower(company#x), %oo%, %in, fa%) + +- SubqueryAlias ilike_all_table + +- View (`ilike_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_all_table WHERE company NOT ILIKE ALL ('microsoft', '%yoo%') +-- !query analysis +Project [company#x] ++- Filter notlikeall(lower(company#x), microsoft, %yoo%) + +- SubqueryAlias ilike_all_table + +- View (`ilike_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_all_table WHERE company NOT ILIKE ALL ('%oo%', 'fA%') +-- !query analysis +Project [company#x] ++- Filter notlikeall(lower(company#x), %oo%, fa%) + +- SubqueryAlias ilike_all_table + +- View (`ilike_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_all_table WHERE NOT company ILIKE ALL ('%oO%', 'fa%') +-- !query analysis +Project [company#x] ++- Filter NOT likeall(lower(company#x), %oo%, fa%) + +- SubqueryAlias ilike_all_table + +- View (`ilike_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_all_table WHERE company ILIKE ALL ('%OO%', NULL) +-- !query analysis +Project [company#x] ++- Filter (ilike(company#x, %OO%, \) AND ilike(company#x, cast(null as string), \)) + +- SubqueryAlias ilike_all_table + +- View (`ilike_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_all_table WHERE company NOT ILIKE ALL ('%Oo%', NULL) +-- !query analysis +Project [company#x] ++- Filter (NOT ilike(company#x, %Oo%, \) AND NOT ilike(company#x, cast(null as string), \)) + +- SubqueryAlias ilike_all_table + +- View (`ilike_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_all_table WHERE company ILIKE ALL (NULL, NULL) +-- !query analysis +Project [company#x] ++- Filter (ilike(company#x, cast(null as string), \) AND ilike(company#x, cast(null as string), \)) + +- SubqueryAlias ilike_all_table + +- View (`ilike_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_all_table WHERE company NOT ILIKE ALL (NULL, NULL) +-- !query analysis +Project [company#x] ++- Filter (NOT ilike(company#x, cast(null as string), \) AND NOT ilike(company#x, cast(null as string), \)) + +- SubqueryAlias ilike_all_table + +- View (`ilike_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_any_table WHERE company ILIKE ALL () +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Expected something between '(' and ')'." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 62, + "fragment" : "ILIKE ALL ()" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ilike-any.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ilike-any.sql.out new file mode 100644 index 0000000000000..1d9d06243f97b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ilike-any.sql.out @@ -0,0 +1,213 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW ilike_any_table AS SELECT * FROM (VALUES + ('Google', '%Oo%'), + ('FaceBook', '%oO%'), + ('linkedIn', '%IN')) + as t1(company, pat) +-- !query analysis +CreateViewCommand `ilike_any_table`, SELECT * FROM (VALUES + ('Google', '%Oo%'), + ('FaceBook', '%oO%'), + ('linkedIn', '%IN')) + as t1(company, pat), false, true, LocalTempView, true + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_any_table WHERE company ILIKE ANY ('%oo%', '%IN', 'fA%') +-- !query analysis +Project [company#x] ++- Filter likeany(lower(company#x), %oo%, %in, fa%) + +- SubqueryAlias ilike_any_table + +- View (`ilike_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_any_table WHERE company ILIKE ANY ('microsoft', '%yoo%') +-- !query analysis +Project [company#x] ++- Filter likeany(lower(company#x), microsoft, %yoo%) + +- SubqueryAlias ilike_any_table + +- View (`ilike_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select + company, + CASE + WHEN company ILIKE ANY ('%oO%', '%IN', 'Fa%') THEN 'Y' + ELSE 'N' + END AS is_available, + CASE + WHEN company ILIKE ANY ('%OO%', 'fa%') OR company ILIKE ANY ('%in', 'MS%') THEN 'Y' + ELSE 'N' + END AS mix +FROM ilike_any_table +-- !query analysis +Project [company#x, CASE WHEN likeany(lower(company#x), %oo%, %in, fa%) THEN Y ELSE N END AS is_available#x, CASE WHEN (likeany(lower(company#x), %oo%, fa%) OR likeany(lower(company#x), %in, ms%)) THEN Y ELSE N END AS mix#x] ++- SubqueryAlias ilike_any_table + +- View (`ilike_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_any_table WHERE company ILIKE ANY ('%zZ%', pat) +-- !query analysis +Project [company#x] ++- Filter (ilike(company#x, %zZ%, \) OR ilike(company#x, pat#x, \)) + +- SubqueryAlias ilike_any_table + +- View (`ilike_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_any_table WHERE company NOT ILIKE ANY ('%oO%', '%iN', 'fa%') +-- !query analysis +Project [company#x] ++- Filter notlikeany(lower(company#x), %oo%, %in, fa%) + +- SubqueryAlias ilike_any_table + +- View (`ilike_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_any_table WHERE company NOT ILIKE ANY ('microsoft', '%yOo%') +-- !query analysis +Project [company#x] ++- Filter notlikeany(lower(company#x), microsoft, %yoo%) + +- SubqueryAlias ilike_any_table + +- View (`ilike_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_any_table WHERE company NOT ILIKE ANY ('%oo%', 'Fa%') +-- !query analysis +Project [company#x] ++- Filter notlikeany(lower(company#x), %oo%, fa%) + +- SubqueryAlias ilike_any_table + +- View (`ilike_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_any_table WHERE NOT company ILIKE ANY ('%OO%', 'fa%') +-- !query analysis +Project [company#x] ++- Filter NOT likeany(lower(company#x), %oo%, fa%) + +- SubqueryAlias ilike_any_table + +- View (`ilike_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_any_table WHERE company ILIKE ANY ('%oO%', NULL) +-- !query analysis +Project [company#x] ++- Filter (ilike(company#x, %oO%, \) OR ilike(company#x, cast(null as string), \)) + +- SubqueryAlias ilike_any_table + +- View (`ilike_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_any_table WHERE company NOT ILIKE ANY ('%oo%', NULL) +-- !query analysis +Project [company#x] ++- Filter (NOT ilike(company#x, %oo%, \) OR NOT ilike(company#x, cast(null as string), \)) + +- SubqueryAlias ilike_any_table + +- View (`ilike_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_any_table WHERE company ILIKE ANY (NULL, NULL) +-- !query analysis +Project [company#x] ++- Filter (ilike(company#x, cast(null as string), \) OR ilike(company#x, cast(null as string), \)) + +- SubqueryAlias ilike_any_table + +- View (`ilike_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_any_table WHERE company NOT ILIKE ANY (NULL, NULL) +-- !query analysis +Project [company#x] ++- Filter (NOT ilike(company#x, cast(null as string), \) OR NOT ilike(company#x, cast(null as string), \)) + +- SubqueryAlias ilike_any_table + +- View (`ilike_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM ilike_any_table WHERE company ILIKE ANY () +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Expected something between '(' and ')'." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 62, + "fragment" : "ILIKE ANY ()" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ilike.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ilike.sql.out new file mode 100644 index 0000000000000..350477ed57a33 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ilike.sql.out @@ -0,0 +1,183 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select null ilike 'a' +-- !query analysis +Project [ilike(cast(null as string), a, \) AS ilike(NULL, a)#x] ++- OneRowRelation + + +-- !query +select 'a' ilike null +-- !query analysis +Project [ilike(a, cast(null as string), \) AS ilike(a, NULL)#x] ++- OneRowRelation + + +-- !query +select null ilike null +-- !query analysis +Project [ilike(cast(null as string), cast(null as string), \) AS ilike(NULL, NULL)#x] ++- OneRowRelation + + +-- !query +select 'a' ilike 'a' +-- !query analysis +Project [ilike(a, a, \) AS ilike(a, a)#x] ++- OneRowRelation + + +-- !query +select 'a' ilike 'b' +-- !query analysis +Project [ilike(a, b, \) AS ilike(a, b)#x] ++- OneRowRelation + + +-- !query +select 'A' ilike 'a' +-- !query analysis +Project [ilike(A, a, \) AS ilike(A, a)#x] ++- OneRowRelation + + +-- !query +select 'a' ilike 'A' +-- !query analysis +Project [ilike(a, A, \) AS ilike(a, A)#x] ++- OneRowRelation + + +-- !query +select 'abdef' ilike 'aBdef' +-- !query analysis +Project [ilike(abdef, aBdef, \) AS ilike(abdef, aBdef)#x] ++- OneRowRelation + + +-- !query +select 'a_%b' ilike 'a\\__b' +-- !query analysis +Project [ilike(a_%b, a\__b, \) AS ilike(a_%b, a\__b)#x] ++- OneRowRelation + + +-- !query +select 'addb' ilike 'A_%b' +-- !query analysis +Project [ilike(addb, A_%b, \) AS ilike(addb, A_%b)#x] ++- OneRowRelation + + +-- !query +select 'abC' ilike 'a%' +-- !query analysis +Project [ilike(abC, a%, \) AS ilike(abC, a%)#x] ++- OneRowRelation + + +-- !query +select 'a\nb' ilike 'a_B' +-- !query analysis +Project [ilike(a +b, a_B, \) AS ilike(a +b, a_B)#x] ++- OneRowRelation + + +-- !query +select '' ilike '' +-- !query analysis +Project [ilike(, , \) AS ilike(, )#x] ++- OneRowRelation + + +-- !query +select 'A' ilike '' +-- !query analysis +Project [ilike(A, , \) AS ilike(A, )#x] ++- OneRowRelation + + +-- !query +select '' ilike 'a' +-- !query analysis +Project [ilike(, a, \) AS ilike(, a)#x] ++- OneRowRelation + + +-- !query +select ilike('\__', '\\\__') +-- !query analysis +Project [ilike(\__, \\__, \) AS ilike(\__, \\__)#x] ++- OneRowRelation + + +-- !query +select ilike('\\\__', '%\\%\%') +-- !query analysis +Project [ilike(\\__, %\%\%, \) AS ilike(\\__, %\%\%)#x] ++- OneRowRelation + + +-- !query +select 'a\u20ACA' ilike '_\u20AC_' +-- !query analysis +Project [ilike(a€A, _€_, \) AS ilike(a€A, _€_)#x] ++- OneRowRelation + + +-- !query +select 'A€a' ilike '_€_' +-- !query analysis +Project [ilike(A€a, _€_, \) AS ilike(A€a, _€_)#x] ++- OneRowRelation + + +-- !query +select 'a€AA' ilike '_\u20AC_a' +-- !query analysis +Project [ilike(a€AA, _€_a, \) AS ilike(a€AA, _€_a)#x] ++- OneRowRelation + + +-- !query +select 'a\u20ACaz' ilike '_€_Z' +-- !query analysis +Project [ilike(a€az, _€_Z, \) AS ilike(a€az, _€_Z)#x] ++- OneRowRelation + + +-- !query +select 'ЀЁЂѺΏỀ' ilike 'ѐёђѻώề' +-- !query analysis +Project [ilike(ЀЁЂѺΏỀ, ѐёђѻώề, \) AS ilike(ЀЁЂѺΏỀ, ѐёђѻώề)#x] ++- OneRowRelation + + +-- !query +select 'Addb' ilike 'a%#%b' escape '#' +-- !query analysis +Project [ilike(Addb, a%#%b, #) AS ilike(Addb, a%#%b)#x] ++- OneRowRelation + + +-- !query +select 'a_%b' ilike 'a%#%B' escape '#' +-- !query analysis +Project [ilike(a_%b, a%#%B, #) AS ilike(a_%b, a%#%B)#x] ++- OneRowRelation + + +-- !query +select 'Addb' ilike 'A%$%b' escape '$' +-- !query analysis +Project [ilike(Addb, A%$%b, $) AS ilike(Addb, A%$%b)#x] ++- OneRowRelation + + +-- !query +select 'a_%b' ilike 'a%+%B' escape '+' +-- !query analysis +Project [ilike(a_%b, a%+%B, +) AS ilike(a_%b, a%+%B)#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/inline-table.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/inline-table.sql.out new file mode 100644 index 0000000000000..9ff237d432493 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/inline-table.sql.out @@ -0,0 +1,222 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select * from values ("one", 1) +-- !query analysis +Project [col1#x, col2#x] ++- LocalRelation [col1#x, col2#x] + + +-- !query +select * from values ("one", 1) as data +-- !query analysis +Project [col1#x, col2#x] ++- SubqueryAlias data + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from values ("one", 1) as data(a, b) +-- !query analysis +Project [a#x, b#x] ++- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select * from values 1, 2, 3 as data(a) +-- !query analysis +Project [a#x] ++- SubqueryAlias data + +- LocalRelation [a#x] + + +-- !query +select * from values ("one", 1), ("two", 2), ("three", null) as data(a, b) +-- !query analysis +Project [a#x, b#x] ++- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select * from values ("one", null), ("two", null) as data(a, b) +-- !query analysis +Project [a#x, b#x] ++- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select * from values ("one", 1), ("two", 2L) as data(a, b) +-- !query analysis +Project [a#x, b#xL] ++- SubqueryAlias data + +- LocalRelation [a#x, b#xL] + + +-- !query +select * from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b) +-- !query analysis +Project [a#x, b#xL] ++- SubqueryAlias data + +- LocalRelation [a#x, b#xL] + + +-- !query +select * from values ("one", 1 as one) as data(a, b) +-- !query analysis +Project [a#x, b#x] ++- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select a from values ("one", current_timestamp) as data(a, b) +-- !query analysis +Project [a#x] ++- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select * from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b) +-- !query analysis +Project [a#x, b#x] ++- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select * from values ("one", 2.0), ("two", 3.0D) as data(a, b) +-- !query analysis +Project [a#x, b#x] ++- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select * from values ("one", rand(5)), ("two", 3.0D) as data(a, b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2304", + "messageParameters" : { + "sqlExpr" : "rand(5)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 36, + "fragment" : "rand(5)" + } ] +} + + +-- !query +select * from values ("one", 2.0), ("two") as data(a, b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2305", + "messageParameters" : { + "numCols" : "2", + "ri" : "1", + "rowSize" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 56, + "fragment" : "values (\"one\", 2.0), (\"two\") as data(a, b)" + } ] +} + + +-- !query +select * from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2303", + "messageParameters" : { + "name" : "b" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 78, + "fragment" : "values (\"one\", array(0, 1)), (\"two\", struct(1, 2)) as data(a, b)" + } ] +} + + +-- !query +select * from values ("one"), ("two") as data(a, b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2305", + "messageParameters" : { + "numCols" : "2", + "ri" : "0", + "rowSize" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 51, + "fragment" : "values (\"one\"), (\"two\") as data(a, b)" + } ] +} + + +-- !query +select * from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`random_not_exist_func`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 53, + "fragment" : "random_not_exist_func(1)" + } ] +} + + +-- !query +select * from values ("one", count(1)), ("two", 2) as data(a, b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2304", + "messageParameters" : { + "sqlExpr" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 37, + "fragment" : "count(1)" + } ] +} + + +-- !query +select * from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b) +-- !query analysis +Project [a#x, b#x] ++- SubqueryAlias data + +- LocalRelation [a#x, b#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/inner-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/inner-join.sql.out new file mode 100644 index 0000000000000..375f889b58f0b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/inner-join.sql.out @@ -0,0 +1,130 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query analysis +CreateViewCommand `t1`, SELECT * FROM VALUES (1) AS GROUPING(a), false, false, LocalTempView, true + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query analysis +CreateViewCommand `t2`, SELECT * FROM VALUES (1) AS GROUPING(a), false, false, LocalTempView, true + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) +-- !query analysis +CreateViewCommand `t3`, SELECT * FROM VALUES (1), (1) AS GROUPING(a), false, false, LocalTempView, true + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) +-- !query analysis +CreateViewCommand `t4`, SELECT * FROM VALUES (1), (1) AS GROUPING(a), false, false, LocalTempView, true + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +CREATE TEMPORARY VIEW ta AS +SELECT a, 'a' AS tag FROM t1 +UNION ALL +SELECT a, 'b' AS tag FROM t2 +-- !query analysis +CreateViewCommand `ta`, SELECT a, 'a' AS tag FROM t1 +UNION ALL +SELECT a, 'b' AS tag FROM t2, false, false, LocalTempView, true + +- Union false, false + :- Project [a#x, a AS tag#x] + : +- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- Project [a#x, b AS tag#x] + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +CREATE TEMPORARY VIEW tb AS +SELECT a, 'a' AS tag FROM t3 +UNION ALL +SELECT a, 'b' AS tag FROM t4 +-- !query analysis +CreateViewCommand `tb`, SELECT a, 'a' AS tag FROM t3 +UNION ALL +SELECT a, 'b' AS tag FROM t4, false, false, LocalTempView, true + +- Union false, false + :- Project [a#x, a AS tag#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- Project [a#x, b AS tag#x] + +- SubqueryAlias t4 + +- View (`t4`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag +-- !query analysis +Project [a#x, tag#x] ++- Join Inner, ((a#x = a#x) AND (tag#x = tag#x)) + :- SubqueryAlias ta + : +- View (`ta`, [a#x,tag#x]) + : +- Project [cast(a#x as int) AS a#x, cast(tag#x as string) AS tag#x] + : +- Union false, false + : :- Project [a#x, a AS tag#x] + : : +- SubqueryAlias t1 + : : +- View (`t1`, [a#x]) + : : +- Project [cast(a#x as int) AS a#x] + : : +- Project [a#x] + : : +- SubqueryAlias GROUPING + : : +- LocalRelation [a#x] + : +- Project [a#x, b AS tag#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias tb + +- View (`tb`, [a#x,tag#x]) + +- Project [cast(a#x as int) AS a#x, cast(tag#x as string) AS tag#x] + +- Union false, false + :- Project [a#x, a AS tag#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- Project [a#x, b AS tag#x] + +- SubqueryAlias t4 + +- View (`t4`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/intersect-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/intersect-all.sql.out new file mode 100644 index 0000000000000..105741b80a20f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/intersect-all.sql.out @@ -0,0 +1,666 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (1, 3), + (2, 3), + (null, null), + (null, null) + AS tab1(k, v) +-- !query analysis +CreateViewCommand `tab1`, SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (1, 3), + (2, 3), + (null, null), + (null, null) + AS tab1(k, v), false, false, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias tab1 + +- LocalRelation [k#x, v#x] + + +-- !query +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (2, 3), + (3, 4), + (null, null), + (null, null) + AS tab2(k, v) +-- !query analysis +CreateViewCommand `tab2`, SELECT * FROM VALUES + (1, 2), + (1, 2), + (2, 3), + (3, 4), + (null, null), + (null, null) + AS tab2(k, v), false, false, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2 +-- !query analysis +Intersect All true +:- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab1 WHERE k = 1 +-- !query analysis +Intersect All true +:- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- Filter (k#x = 1) + +- SubqueryAlias tab1 + +- View (`tab1`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab1 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM tab1 WHERE k > 2 +INTERSECT ALL +SELECT * FROM tab2 +-- !query analysis +Intersect All true +:- Project [k#x, v#x] +: +- Filter (k#x > 2) +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2 WHERE k > 3 +-- !query analysis +Intersect All true +:- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- Filter (k#x > 3) + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM tab1 +INTERSECT ALL +SELECT CAST(1 AS BIGINT), CAST(2 AS BIGINT) +-- !query analysis +Intersect All true +:- Project [cast(k#x as bigint) AS k#xL, cast(v#x as bigint) AS v#xL] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [cast(1 as bigint) AS CAST(1 AS BIGINT)#xL, cast(2 as bigint) AS CAST(2 AS BIGINT)#xL] + +- OneRowRelation + + +-- !query +SELECT * FROM tab1 +INTERSECT ALL +SELECT array(1), 2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"ARRAY\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "INTERSECT ALL", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 51, + "fragment" : "SELECT * FROM tab1\nINTERSECT ALL\nSELECT array(1), 2" + } ] +} + + +-- !query +SELECT k FROM tab1 +INTERSECT ALL +SELECT k, v FROM tab2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_COLUMNS_MISMATCH", + "sqlState" : "42826", + "messageParameters" : { + "firstNumColumns" : "1", + "invalidNumColumns" : "2", + "invalidOrdinalNum" : "second", + "operator" : "INTERSECT ALL" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT k FROM tab1\nINTERSECT ALL\nSELECT k, v FROM tab2" + } ] +} + + +-- !query +SELECT * FROM tab2 +INTERSECT ALL +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2 +-- !query analysis +Intersect All true +:- Intersect All true +: :- Project [k#x, v#x] +: : +- SubqueryAlias tab2 +: : +- View (`tab2`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab2 +: : +- LocalRelation [k#x, v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM tab1 +EXCEPT +SELECT * FROM tab2 +UNION ALL +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2 +-- !query analysis +Union false, false +:- Except false +: :- Project [k#x, v#x] +: : +- SubqueryAlias tab1 +: : +- View (`tab1`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab1 +: : +- LocalRelation [k#x, v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab2 +: +- View (`tab2`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab2 +: +- LocalRelation [k#x, v#x] ++- Intersect All true + :- Project [k#x, v#x] + : +- SubqueryAlias tab1 + : +- View (`tab1`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias tab1 + : +- LocalRelation [k#x, v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM tab1 +EXCEPT +SELECT * FROM tab2 +EXCEPT +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2 +-- !query analysis +Except false +:- Except false +: :- Project [k#x, v#x] +: : +- SubqueryAlias tab1 +: : +- View (`tab1`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab1 +: : +- LocalRelation [k#x, v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab2 +: +- View (`tab2`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab2 +: +- LocalRelation [k#x, v#x] ++- Intersect All true + :- Project [k#x, v#x] + : +- SubqueryAlias tab1 + : +- View (`tab1`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias tab1 + : +- LocalRelation [k#x, v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +( + ( + ( + SELECT * FROM tab1 + EXCEPT + SELECT * FROM tab2 + ) + EXCEPT + SELECT * FROM tab1 + ) + INTERSECT ALL + SELECT * FROM tab2 +) +-- !query analysis +Intersect All true +:- Except false +: :- Except false +: : :- Project [k#x, v#x] +: : : +- SubqueryAlias tab1 +: : : +- View (`tab1`, [k#x,v#x]) +: : : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : : +- Project [k#x, v#x] +: : : +- SubqueryAlias tab1 +: : : +- LocalRelation [k#x, v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab2 +: : +- View (`tab2`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab2 +: : +- LocalRelation [k#x, v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * +FROM (SELECT tab1.k, + tab2.v + FROM tab1 + JOIN tab2 + ON tab1.k = tab2.k) +INTERSECT ALL +SELECT * +FROM (SELECT tab1.k, + tab2.v + FROM tab1 + JOIN tab2 + ON tab1.k = tab2.k) +-- !query analysis +Intersect All true +:- Project [k#x, v#x] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [k#x, v#x] +: +- Join Inner, (k#x = k#x) +: :- SubqueryAlias tab1 +: : +- View (`tab1`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab1 +: : +- LocalRelation [k#x, v#x] +: +- SubqueryAlias tab2 +: +- View (`tab2`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab2 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [k#x, v#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias tab1 + : +- View (`tab1`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias tab1 + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * +FROM (SELECT tab1.k, + tab2.v + FROM tab1 + JOIN tab2 + ON tab1.k = tab2.k) +INTERSECT ALL +SELECT * +FROM (SELECT tab2.v AS k, + tab1.k AS v + FROM tab1 + JOIN tab2 + ON tab1.k = tab2.k) +-- !query analysis +Intersect All true +:- Project [k#x, v#x] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [k#x, v#x] +: +- Join Inner, (k#x = k#x) +: :- SubqueryAlias tab1 +: : +- View (`tab1`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab1 +: : +- LocalRelation [k#x, v#x] +: +- SubqueryAlias tab2 +: +- View (`tab2`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab2 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [v#x AS k#x, k#x AS v#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias tab1 + : +- View (`tab1`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias tab1 + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT v FROM tab1 GROUP BY v +INTERSECT ALL +SELECT k FROM tab2 GROUP BY k +-- !query analysis +Intersect All true +:- Aggregate [v#x], [v#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Aggregate [k#x], [k#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SET spark.sql.legacy.setopsPrecedence.enabled= true +-- !query analysis +SetCommand (spark.sql.legacy.setopsPrecedence.enabled,Some(true)) + + +-- !query +SELECT * FROM tab1 +EXCEPT +SELECT * FROM tab2 +UNION ALL +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2 +-- !query analysis +Intersect All true +:- Union false, false +: :- Except false +: : :- Project [k#x, v#x] +: : : +- SubqueryAlias tab1 +: : : +- View (`tab1`, [k#x,v#x]) +: : : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : : +- Project [k#x, v#x] +: : : +- SubqueryAlias tab1 +: : : +- LocalRelation [k#x, v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab2 +: : +- View (`tab2`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab2 +: : +- LocalRelation [k#x, v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM tab1 +EXCEPT +SELECT * FROM tab2 +UNION ALL +SELECT * FROM tab1 +INTERSECT +SELECT * FROM tab2 +-- !query analysis +Intersect false +:- Union false, false +: :- Except false +: : :- Project [k#x, v#x] +: : : +- SubqueryAlias tab1 +: : : +- View (`tab1`, [k#x,v#x]) +: : : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : : +- Project [k#x, v#x] +: : : +- SubqueryAlias tab1 +: : : +- LocalRelation [k#x, v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab2 +: : +- View (`tab2`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab2 +: : +- LocalRelation [k#x, v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SET spark.sql.legacy.setopsPrecedence.enabled = false +-- !query analysis +SetCommand (spark.sql.legacy.setopsPrecedence.enabled,Some(false)) + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW tab3 AS VALUES (decimal(1)), (decimal(2)) tbl3(v) +-- !query analysis +CreateViewCommand `tab3`, VALUES (decimal(1)), (decimal(2)) tbl3(v), false, true, LocalTempView, true + +- SubqueryAlias tbl3 + +- LocalRelation [v#x] + + +-- !query +SELECT t.v FROM ( + SELECT v FROM tab3 + INTERSECT + SELECT v + v AS v FROM tab3 +) t +-- !query analysis +Project [v#x] ++- SubqueryAlias t + +- Intersect false + :- Project [cast(v#x as decimal(11,0)) AS v#x] + : +- Project [v#x] + : +- SubqueryAlias tab3 + : +- View (`tab3`, [v#x]) + : +- Project [cast(v#x as decimal(10,0)) AS v#x] + : +- SubqueryAlias tbl3 + : +- LocalRelation [v#x] + +- Project [(v#x + v#x) AS v#x] + +- SubqueryAlias tab3 + +- View (`tab3`, [v#x]) + +- Project [cast(v#x as decimal(10,0)) AS v#x] + +- SubqueryAlias tbl3 + +- LocalRelation [v#x] + + +-- !query +SELECT SUM(t.v) FROM ( + SELECT v FROM tab3 + INTERSECT + SELECT v + v AS v FROM tab3 +) t +-- !query analysis +Aggregate [sum(v#x) AS sum(v)#x] ++- SubqueryAlias t + +- Intersect false + :- Project [cast(v#x as decimal(11,0)) AS v#x] + : +- Project [v#x] + : +- SubqueryAlias tab3 + : +- View (`tab3`, [v#x]) + : +- Project [cast(v#x as decimal(10,0)) AS v#x] + : +- SubqueryAlias tbl3 + : +- LocalRelation [v#x] + +- Project [(v#x + v#x) AS v#x] + +- SubqueryAlias tab3 + +- View (`tab3`, [v#x]) + +- Project [cast(v#x as decimal(10,0)) AS v#x] + +- SubqueryAlias tbl3 + +- LocalRelation [v#x] + + +-- !query +DROP VIEW IF EXISTS tab1 +-- !query analysis +DropTempViewCommand tab1 + + +-- !query +DROP VIEW IF EXISTS tab2 +-- !query analysis +DropTempViewCommand tab2 + + +-- !query +DROP VIEW IF EXISTS tab3 +-- !query analysis +DropTempViewCommand tab3 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 new file mode 100644 index 0000000000000..19abe19e4c656 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/interval.sql.out @@ -0,0 +1,3183 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select interval 4 month 2 weeks 3 microseconds * 1.5 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 4 month 2 weeks 3 microseconds" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "interval 4 month 2 weeks 3 microseconds" + } ] +} + + +-- !query +select interval 2 years 4 months +-- !query analysis +Project [INTERVAL '2-4' YEAR TO MONTH AS INTERVAL '2-4' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval 2 weeks 3 microseconds * 1.5 +-- !query analysis +Project [(INTERVAL '14 00:00:00.000003' DAY TO SECOND * 1.5) AS (INTERVAL '14 00:00:00.000003' DAY TO SECOND * 1.5)#x] ++- OneRowRelation + + +-- !query +select (timestamp'2019-10-15' - timestamp'2019-10-14') / 1.5 +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select interval 2147483647 month * 2 +-- !query analysis +Project [(INTERVAL '2147483647' MONTH * 2) AS (INTERVAL '2147483647' MONTH * 2)#x] ++- OneRowRelation + + +-- !query +select interval 2147483647 month / 0.5 +-- !query analysis +Project [(INTERVAL '2147483647' MONTH / 0.5) AS (INTERVAL '2147483647' MONTH / 0.5)#x] ++- OneRowRelation + + +-- !query +select interval 2147483647 day * 2 +-- !query analysis +java.lang.ArithmeticException +long overflow + + +-- !query +select interval 2147483647 day / 0.5 +-- !query analysis +java.lang.ArithmeticException +long overflow + + +-- !query +select interval 2 second * '2' +-- !query analysis +Project [(INTERVAL '02' SECOND * cast(2 as double)) AS (INTERVAL '02' SECOND * 2)#x] ++- OneRowRelation + + +-- !query +select interval 2 second / '2' +-- !query analysis +Project [(INTERVAL '02' SECOND / cast(2 as double)) AS (INTERVAL '02' SECOND / 2)#x] ++- OneRowRelation + + +-- !query +select interval 2 year * '2' +-- !query analysis +Project [(INTERVAL '2' YEAR * cast(2 as double)) AS (INTERVAL '2' YEAR * 2)#x] ++- OneRowRelation + + +-- !query +select interval 2 year / '2' +-- !query analysis +Project [(INTERVAL '2' YEAR / cast(2 as double)) AS (INTERVAL '2' YEAR / 2)#x] ++- OneRowRelation + + +-- !query +select interval 2 second * 'a' +-- !query analysis +Project [(INTERVAL '02' SECOND * cast(a as double)) AS (INTERVAL '02' SECOND * a)#x] ++- OneRowRelation + + +-- !query +select interval 2 second / 'a' +-- !query analysis +Project [(INTERVAL '02' SECOND / cast(a as double)) AS (INTERVAL '02' SECOND / a)#x] ++- OneRowRelation + + +-- !query +select interval 2 year * 'a' +-- !query analysis +Project [(INTERVAL '2' YEAR * cast(a as double)) AS (INTERVAL '2' YEAR * a)#x] ++- OneRowRelation + + +-- !query +select interval 2 year / 'a' +-- !query analysis +Project [(INTERVAL '2' YEAR / cast(a as double)) AS (INTERVAL '2' YEAR / a)#x] ++- OneRowRelation + + +-- !query +select '2' * interval 2 second +-- !query analysis +Project [(INTERVAL '02' SECOND * cast(2 as double)) AS (INTERVAL '02' SECOND * 2)#x] ++- OneRowRelation + + +-- !query +select '2' * interval 2 year +-- !query analysis +Project [(INTERVAL '2' YEAR * cast(2 as double)) AS (INTERVAL '2' YEAR * 2)#x] ++- OneRowRelation + + +-- !query +select 'a' * interval 2 second +-- !query analysis +Project [(INTERVAL '02' SECOND * cast(a as double)) AS (INTERVAL '02' SECOND * a)#x] ++- OneRowRelation + + +-- !query +select 'a' * interval 2 year +-- !query analysis +Project [(INTERVAL '2' YEAR * cast(a as double)) AS (INTERVAL '2' YEAR * a)#x] ++- OneRowRelation + + +-- !query +select '2' / interval 2 second +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"INTERVAL SECOND\"", + "sqlExpr" : "\"(2 / INTERVAL '02' SECOND)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'2' / interval 2 second" + } ] +} + + +-- !query +select '2' / interval 2 year +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "'2' / interval 2 year" + } ] +} + + +-- !query +select interval '2 seconds' / 0 +-- !query analysis +Project [(INTERVAL '02' SECOND / 0) AS (INTERVAL '02' SECOND / 0)#x] ++- OneRowRelation + + +-- !query +select interval '2 seconds' / null +-- !query analysis +Project [(INTERVAL '02' SECOND / cast(null as double)) AS (INTERVAL '02' SECOND / NULL)#x] ++- OneRowRelation + + +-- !query +select interval '2 seconds' * null +-- !query analysis +Project [(INTERVAL '02' SECOND * cast(null as double)) AS (INTERVAL '02' SECOND * NULL)#x] ++- OneRowRelation + + +-- !query +select null * interval '2 seconds' +-- !query analysis +Project [(INTERVAL '02' SECOND * cast(null as double)) AS (INTERVAL '02' SECOND * NULL)#x] ++- OneRowRelation + + +-- !query +select interval '2' year / 0 +-- !query analysis +Project [(INTERVAL '2' YEAR / 0) AS (INTERVAL '2' YEAR / 0)#x] ++- OneRowRelation + + +-- !query +select interval '2' year / null +-- !query analysis +Project [(INTERVAL '2' YEAR / cast(null as double)) AS (INTERVAL '2' YEAR / NULL)#x] ++- OneRowRelation + + +-- !query +select interval '2' year * null +-- !query analysis +Project [(INTERVAL '2' YEAR * cast(null as double)) AS (INTERVAL '2' YEAR * NULL)#x] ++- OneRowRelation + + +-- !query +select null * interval '2' year +-- !query analysis +Project [(INTERVAL '2' YEAR * cast(null as double)) AS (INTERVAL '2' YEAR * NULL)#x] ++- OneRowRelation + + +-- !query +select 2 / interval '2' year +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "2 / interval '2' year" + } ] +} + + +-- !query +select 2 / interval '2' hour +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"INTERVAL HOUR\"", + "sqlExpr" : "\"(2 / INTERVAL '02' HOUR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "2 / interval '2' hour" + } ] +} + + +-- !query +select null / interval '2' year +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(NULL / INTERVAL '2' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "null / interval '2' year" + } ] +} + + +-- !query +select null / interval '2' hour +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"INTERVAL HOUR\"", + "sqlExpr" : "\"(NULL / INTERVAL '02' HOUR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "null / interval '2' hour" + } ] +} + + +-- !query +select -interval '-1 month 1 day -1 second' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '-1 month 1 day -1 second'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 43, + "fragment" : "interval '-1 month 1 day -1 second'" + } ] +} + + +-- !query +select -interval '-1 year 1 month' +-- !query analysis +Project [-INTERVAL '-0-11' YEAR TO MONTH AS (- INTERVAL '-0-11' YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select -interval '-1 day 1 hour -1 minute 1 second' +-- !query analysis +Project [-INTERVAL '-0 23:00:59' DAY TO SECOND AS (- INTERVAL '-0 23:00:59' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select -interval -1 month 1 day -1 second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval -1 month 1 day -1 second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 41, + "fragment" : "interval -1 month 1 day -1 second" + } ] +} + + +-- !query +select -interval -1 year 1 month +-- !query analysis +Project [-INTERVAL '-0-11' YEAR TO MONTH AS (- INTERVAL '-0-11' YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select -interval -1 day 1 hour -1 minute 1 second +-- !query analysis +Project [-INTERVAL '-0 23:00:59' DAY TO SECOND AS (- INTERVAL '-0 23:00:59' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select +interval '-1 month 1 day -1 second' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '-1 month 1 day -1 second'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 43, + "fragment" : "interval '-1 month 1 day -1 second'" + } ] +} + + +-- !query +select +interval '-1 year 1 month' +-- !query analysis +Project [positive(INTERVAL '-0-11' YEAR TO MONTH) AS (+ INTERVAL '-0-11' YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select +interval '-1 day 1 hour -1 minute 1 second' +-- !query analysis +Project [positive(INTERVAL '-0 23:00:59' DAY TO SECOND) AS (+ INTERVAL '-0 23:00:59' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select +interval -1 month 1 day -1 second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval -1 month 1 day -1 second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 41, + "fragment" : "interval -1 month 1 day -1 second" + } ] +} + + +-- !query +select +interval -1 year 1 month +-- !query analysis +Project [positive(INTERVAL '-0-11' YEAR TO MONTH) AS (+ INTERVAL '-0-11' YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +select +interval -1 day 1 hour -1 minute 1 second +-- !query analysis +Project [positive(INTERVAL '-0 23:00:59' DAY TO SECOND) AS (+ INTERVAL '-0 23:00:59' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +select interval -'1-1' year to month +-- !query analysis +Project [INTERVAL '-1-1' YEAR TO MONTH AS INTERVAL '-1-1' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval -'-1-1' year to month +-- !query analysis +Project [INTERVAL '1-1' YEAR TO MONTH AS INTERVAL '1-1' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval +'-1-1' year to month +-- !query analysis +Project [INTERVAL '-1-1' YEAR TO MONTH AS INTERVAL '-1-1' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval - '1 2:3:4.001' day to second +-- !query analysis +Project [INTERVAL '-1 02:03:04.001' DAY TO SECOND AS INTERVAL '-1 02:03:04.001' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval +'1 2:3:4.001' day to second +-- !query analysis +Project [INTERVAL '1 02:03:04.001' DAY TO SECOND AS INTERVAL '1 02:03:04.001' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval -'-1 2:3:4.001' day to second +-- !query analysis +Project [INTERVAL '1 02:03:04.001' DAY TO SECOND AS INTERVAL '1 02:03:04.001' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval -'1' year +-- !query analysis +Project [INTERVAL '-1' YEAR AS INTERVAL '-1' YEAR#x] ++- OneRowRelation + + +-- !query +select interval -'-1' year +-- !query analysis +Project [INTERVAL '1' YEAR AS INTERVAL '1' YEAR#x] ++- OneRowRelation + + +-- !query +select interval -'11' month +-- !query analysis +Project [INTERVAL '-11' MONTH AS INTERVAL '-11' MONTH#x] ++- OneRowRelation + + +-- !query +select interval -'-11' month +-- !query analysis +Project [INTERVAL '11' MONTH AS INTERVAL '11' MONTH#x] ++- OneRowRelation + + +-- !query +select interval -'1' day +-- !query analysis +Project [INTERVAL '-1' DAY AS INTERVAL '-1' DAY#x] ++- OneRowRelation + + +-- !query +select interval -'-1' day +-- !query analysis +Project [INTERVAL '1' DAY AS INTERVAL '1' DAY#x] ++- OneRowRelation + + +-- !query +select interval -'23' hour +-- !query analysis +Project [INTERVAL '-23' HOUR AS INTERVAL '-23' HOUR#x] ++- OneRowRelation + + +-- !query +select interval -'-23' hour +-- !query analysis +Project [INTERVAL '23' HOUR AS INTERVAL '23' HOUR#x] ++- OneRowRelation + + +-- !query +select interval -'59' minute +-- !query analysis +Project [INTERVAL '-59' MINUTE AS INTERVAL '-59' MINUTE#x] ++- OneRowRelation + + +-- !query +select interval -'-59' minute +-- !query analysis +Project [INTERVAL '59' MINUTE AS INTERVAL '59' MINUTE#x] ++- OneRowRelation + + +-- !query +select interval -'59' second +-- !query analysis +Project [INTERVAL '-59' SECOND AS INTERVAL '-59' SECOND#x] ++- OneRowRelation + + +-- !query +select interval -'-59' second +-- !query analysis +Project [INTERVAL '59' SECOND AS INTERVAL '59' SECOND#x] ++- OneRowRelation + + +-- !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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select make_dt_interval(1) +-- !query analysis +Project [make_dt_interval(1, 0, 0, 0.000000) AS make_dt_interval(1, 0, 0, 0.000000)#x] ++- OneRowRelation + + +-- !query +select make_dt_interval(1, 2) +-- !query analysis +Project [make_dt_interval(1, 2, 0, 0.000000) AS make_dt_interval(1, 2, 0, 0.000000)#x] ++- OneRowRelation + + +-- !query +select make_dt_interval(1, 2, 3) +-- !query analysis +Project [make_dt_interval(1, 2, 3, 0.000000) AS make_dt_interval(1, 2, 3, 0.000000)#x] ++- OneRowRelation + + +-- !query +select make_dt_interval(1, 2, 3, 4.005006) +-- !query analysis +Project [make_dt_interval(1, 2, 3, cast(4.005006 as decimal(18,6))) AS make_dt_interval(1, 2, 3, 4.005006)#x] ++- OneRowRelation + + +-- !query +select make_dt_interval(1, 0, 0, 123456789012.123456) +-- !query analysis +Project [make_dt_interval(1, 0, 0, 123456789012.123456) AS make_dt_interval(1, 0, 0, 123456789012.123456)#x] ++- OneRowRelation + + +-- !query +select make_dt_interval(2147483647) +-- !query analysis +Project [make_dt_interval(2147483647, 0, 0, 0.000000) AS make_dt_interval(2147483647, 0, 0, 0.000000)#x] ++- OneRowRelation + + +-- !query +select make_ym_interval(1) +-- !query analysis +Project [make_ym_interval(1, 0) AS make_ym_interval(1, 0)#x] ++- OneRowRelation + + +-- !query +select make_ym_interval(1, 2) +-- !query analysis +Project [make_ym_interval(1, 2) AS make_ym_interval(1, 2)#x] ++- OneRowRelation + + +-- !query +select make_ym_interval(0, 1) +-- !query analysis +Project [make_ym_interval(0, 1) AS make_ym_interval(0, 1)#x] ++- OneRowRelation + + +-- !query +select make_ym_interval(178956970, 7) +-- !query analysis +Project [make_ym_interval(178956970, 7) AS make_ym_interval(178956970, 7)#x] ++- OneRowRelation + + +-- !query +select make_ym_interval(178956970, 8) +-- !query analysis +Project [make_ym_interval(178956970, 8) AS make_ym_interval(178956970, 8)#x] ++- OneRowRelation + + +-- !query +select make_ym_interval(-178956970, -8) +-- !query analysis +Project [make_ym_interval(-178956970, -8) AS make_ym_interval(-178956970, -8)#x] ++- OneRowRelation + + +-- !query +select make_ym_interval(-178956970, -9) +-- !query analysis +Project [make_ym_interval(-178956970, -9) AS make_ym_interval(-178956970, -9)#x] ++- OneRowRelation + + +-- !query +select cast('1 second' as interval) +-- !query analysis +Project [cast(1 second as interval) AS CAST(1 second AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +select cast('+1 second' as interval) +-- !query analysis +Project [cast(+1 second as interval) AS CAST(+1 second AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +select cast('-1 second' as interval) +-- !query analysis +Project [cast(-1 second as interval) AS CAST(-1 second AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +select cast('+ 1 second' as interval) +-- !query analysis +Project [cast(+ 1 second as interval) AS CAST(+ 1 second AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +select cast('- 1 second' as interval) +-- !query analysis +Project [cast(- 1 second as interval) AS CAST(- 1 second AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +select cast('- -1 second' as interval) +-- !query analysis +Project [cast(- -1 second as interval) AS CAST(- -1 second AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +select cast('- +1 second' as interval) +-- !query analysis +Project [cast(- +1 second as interval) AS CAST(- +1 second AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +select interval 13.123456789 seconds, interval -13.123456789 second +-- !query analysis +Project [INTERVAL '13.123456' SECOND AS INTERVAL '13.123456' SECOND#x, INTERVAL '-13.123456' SECOND AS INTERVAL '-13.123456' SECOND#x] ++- OneRowRelation + + +-- !query +select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond" + } ] +} + + +-- !query +select interval 1 year 2 month +-- !query analysis +Project [INTERVAL '1-2' YEAR TO MONTH AS INTERVAL '1-2' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval 4 day 5 hour 6 minute 7 seconds +-- !query analysis +Project [INTERVAL '4 05:06:07' DAY TO SECOND AS INTERVAL '4 05:06:07' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval 3 week 8 millisecond 9 microsecond +-- !query analysis +Project [INTERVAL '21 00:00:00.008009' DAY TO SECOND AS INTERVAL '21 00:00:00.008009' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 92, + "fragment" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second" + } ] +} + + +-- !query +select interval '30' year '25' month +-- !query analysis +Project [INTERVAL '32-1' YEAR TO MONTH AS INTERVAL '32-1' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval '-100' day '40' hour '80' minute '299.889987299' second +-- !query analysis +Project [INTERVAL '-98 06:35:00.110013' DAY TO SECOND AS INTERVAL '-98 06:35:00.110013' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '0-0' year to month +-- !query analysis +Project [INTERVAL '0-0' YEAR TO MONTH AS INTERVAL '0-0' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval '0 0:0:0' day to second +-- !query analysis +Project [INTERVAL '0 00:00:00' DAY TO SECOND AS INTERVAL '0 00:00:00' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '0 0:0:0.1' day to second +-- !query analysis +Project [INTERVAL '0 00:00:00.1' DAY TO SECOND AS INTERVAL '0 00:00:00.1' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '10-9' year to month +-- !query analysis +Project [INTERVAL '10-9' YEAR TO MONTH AS INTERVAL '10-9' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval '20 15' day to hour +-- !query analysis +Project [INTERVAL '20 15' DAY TO HOUR AS INTERVAL '20 15' DAY TO HOUR#x] ++- OneRowRelation + + +-- !query +select interval '20 15:40' day to minute +-- !query analysis +Project [INTERVAL '20 15:40' DAY TO MINUTE AS INTERVAL '20 15:40' DAY TO MINUTE#x] ++- OneRowRelation + + +-- !query +select interval '20 15:40:32.99899999' day to second +-- !query analysis +Project [INTERVAL '20 15:40:32.998999' DAY TO SECOND AS INTERVAL '20 15:40:32.998999' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '15:40' hour to minute +-- !query analysis +Project [INTERVAL '15:40' HOUR TO MINUTE AS INTERVAL '15:40' HOUR TO MINUTE#x] ++- OneRowRelation + + +-- !query +select interval '15:40:32.99899999' hour to second +-- !query analysis +Project [INTERVAL '15:40:32.998999' HOUR TO SECOND AS INTERVAL '15:40:32.998999' HOUR TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '40:32.99899999' minute to second +-- !query analysis +Project [INTERVAL '40:32.998999' MINUTE TO SECOND AS INTERVAL '40:32.998999' MINUTE TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '40:32' minute to second +-- !query analysis +Project [INTERVAL '40:32' MINUTE TO SECOND AS INTERVAL '40:32' MINUTE TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval 30 day day +-- !query analysis +Project [INTERVAL '30' DAY AS day#x] ++- OneRowRelation + + +-- !query +select interval 30 days days +-- !query analysis +Project [INTERVAL '30' DAY AS days#x] ++- OneRowRelation + + +-- !query +select interval '20 15:40:32.99899999' day to hour +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 50, + "fragment" : "'20 15:40:32.99899999' day to hour" + } ] +} + + +-- !query +select interval '20 15:40:32.99899999' day to minute +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE` when cast to interval day to minute: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 52, + "fragment" : "'20 15:40:32.99899999' day to minute" + } ] +} + + +-- !query +select interval '15:40:32.99899999' hour to minute +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 50, + "fragment" : "'15:40:32.99899999' hour to minute" + } ] +} + + +-- !query +select interval '15:40.99899999' hour to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 47, + "fragment" : "'15:40.99899999' hour to second" + } ] +} + + +-- !query +select interval '15:40' hour to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 38, + "fragment" : "'15:40' hour to second" + } ] +} + + +-- !query +select interval '20 40:32.99899999' minute to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 20 40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 52, + "fragment" : "'20 40:32.99899999' minute to second" + } ] +} + + +-- !query +select interval 10 nanoseconds +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0062", + "messageParameters" : { + "msg" : "Error parsing ' 10 nanoseconds' to interval, invalid unit 'nanoseconds'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 30, + "fragment" : "10 nanoseconds" + } ] +} + + +-- !query +select map(1, interval 1 day, 2, interval 3 week) +-- !query analysis +Project [map(1, INTERVAL '1' DAY, 2, INTERVAL '21' DAY) AS map(1, INTERVAL '1' DAY, 2, INTERVAL '21' DAY)#x] ++- OneRowRelation + + +-- !query +select map(1, interval 1 day, 2, interval 2 day) +-- !query analysis +Project [map(1, INTERVAL '1' DAY, 2, INTERVAL '2' DAY) AS map(1, INTERVAL '1' DAY, 2, INTERVAL '2' DAY)#x] ++- OneRowRelation + + +-- !query +select map(1, interval 1 year, 2, interval 2 month) +-- !query analysis +Project [map(1, cast(INTERVAL '1' YEAR as interval year to month), 2, cast(INTERVAL '2' MONTH as interval year to month)) AS map(1, INTERVAL '1' YEAR, 2, INTERVAL '2' MONTH)#x] ++- OneRowRelation + + +-- !query +select map(1, interval 1 month, 2, interval 2 month) +-- !query analysis +Project [map(1, INTERVAL '1' MONTH, 2, INTERVAL '2' MONTH) AS map(1, INTERVAL '1' MONTH, 2, INTERVAL '2' MONTH)#x] ++- OneRowRelation + + +-- !query +select map(1, interval 1 week, 2, interval 2 day) +-- !query analysis +Project [map(1, INTERVAL '7' DAY, 2, INTERVAL '2' DAY) AS map(1, INTERVAL '7' DAY, 2, INTERVAL '2' DAY)#x] ++- OneRowRelation + + +-- !query +select map(1, interval 2 millisecond, 3, interval 3 microsecond) +-- !query analysis +Project [map(1, INTERVAL '00.002' SECOND, 3, INTERVAL '00.000003' SECOND) AS map(1, INTERVAL '00.002' SECOND, 3, INTERVAL '00.000003' SECOND)#x] ++- OneRowRelation + + +-- !query +select interval 'interval 3 year 1 month' +-- !query analysis +Project [INTERVAL '3-1' YEAR TO MONTH AS INTERVAL '3-1' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval '3 year 1 month' +-- !query analysis +Project [INTERVAL '3-1' YEAR TO MONTH AS INTERVAL '3-1' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +SELECT interval 'interval 2 weeks 2 days 1 hour 3 minutes 2 seconds 100 millisecond 200 microseconds' +-- !query analysis +Project [INTERVAL '16 01:03:02.1002' DAY TO SECOND AS INTERVAL '16 01:03:02.1002' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +SELECT interval '2 weeks 2 days 1 hour 3 minutes 2 seconds 100 millisecond 200 microseconds' +-- !query analysis +Project [INTERVAL '16 01:03:02.1002' DAY TO SECOND AS INTERVAL '16 01:03:02.1002' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval 1 fake_unit +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'fake_unit'", + "hint" : "" + } +} + + +-- !query +select interval 1 year to month +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0027", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 31, + "fragment" : "1 year to month" + } ] +} + + +-- !query +select interval '1' year to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0028", + "messageParameters" : { + "from" : "year", + "to" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 34, + "fragment" : "'1' year to second" + } ] +} + + +-- !query +select interval '10-9' year to month '2-1' year to month +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "interval '10-9' year to month '2-1' year to month" + } ] +} + + +-- !query +select interval '10-9' year to month '12:11:10' hour to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "interval '10-9' year to month '12:11:10' hour to second" + } ] +} + + +-- !query +select interval '1 15:11' day to minute '12:11:10' hour to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 65, + "fragment" : "interval '1 15:11' day to minute '12:11:10' hour to second" + } ] +} + + +-- !query +select interval 1 year '2-1' year to month +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "interval 1 year '2-1' year to month" + } ] +} + + +-- !query +select interval 1 year '12:11:10' hour to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "interval 1 year '12:11:10' hour to second" + } ] +} + + +-- !query +select interval '10-9' year to month '1' year +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "interval '10-9' year to month '1' year" + } ] +} + + +-- !query +select interval '12:11:10' hour to second '1' year +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "interval '12:11:10' hour to second '1' year" + } ] +} + + +-- !query +select interval (-30) day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval (-30)" + } ] +} + + +-- !query +select interval (a + 1) day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "interval (a + 1)" + } ] +} + + +-- !query +select interval 30 day day day +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'day'", + "hint" : ": extra input 'day'" + } +} + + +-- !query +select interval (-30) days +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval (-30)" + } ] +} + + +-- !query +select interval (a + 1) days +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "interval (a + 1)" + } ] +} + + +-- !query +select interval 30 days days days +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'days'", + "hint" : ": extra input 'days'" + } +} + + +-- !query +SELECT INTERVAL '178956970-7' YEAR TO MONTH +-- !query analysis +Project [INTERVAL '178956970-7' YEAR TO MONTH AS INTERVAL '178956970-7' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '178956970-8' YEAR TO MONTH +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Error parsing interval year-month string: integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 43, + "fragment" : "'178956970-8' YEAR TO MONTH" + } ] +} + + +-- !query +SELECT INTERVAL '-178956970-8' YEAR TO MONTH +-- !query analysis +Project [INTERVAL '-178956970-8' YEAR TO MONTH AS INTERVAL '-178956970-8' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL -'178956970-8' YEAR TO MONTH +-- !query analysis +Project [INTERVAL '-178956970-8' YEAR TO MONTH AS INTERVAL '-178956970-8' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select + interval '2-2' year to month + interval '3' month, + interval '2' year - interval '3-3' year to month, + interval '99 11:22:33.123456789' day to second + interval '10 9:8' day to minute, + interval '22:33.123456789' minute to second - interval '10' day +-- !query analysis +Project [(INTERVAL '2-2' YEAR TO MONTH + cast(INTERVAL '3' MONTH as interval year to month)) AS (INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' MONTH)#x, (cast(INTERVAL '2' YEAR as interval year to month) - INTERVAL '3-3' YEAR TO MONTH) AS (INTERVAL '2' YEAR - INTERVAL '3-3' YEAR TO MONTH)#x, (INTERVAL '99 11:22:33.123456' DAY TO SECOND + cast(INTERVAL '10 09:08' DAY TO MINUTE as interval day to second)) AS (INTERVAL '99 11:22:33.123456' DAY TO SECOND + INTERVAL '10 09:08' DAY TO MINUTE)#x, (cast(INTERVAL '22:33.123456' MINUTE TO SECOND as interval day to second) - cast(INTERVAL '10' DAY as interval day to second)) AS (INTERVAL '22:33.123456' MINUTE TO SECOND - INTERVAL '10' DAY)#x] ++- OneRowRelation + + +-- !query +select + interval '2' year + '3-3 year to month', + interval '2' year - '3 month', + '3-2 year to month' + interval '2-2' year to month, + '3 year' - interval '2-2' year to month, + interval '99 11:22:33.123456789' day to second + '12:12 hour to second', + interval '99 11:22:33.123456789' day to second - '12 hour', + '4 day' + interval '10' day, + '4 22 day to hour' - interval '10' day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3 year to month)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 48, + "fragment" : "interval '2' year + '3-3 year to month'" + } ] +} + + +-- !query +select + interval '2' year + null, + interval '2' year - null, + interval '2' hour + null, + interval '2' hour - null, + null + interval '2' year, + null - interval '2' year, + null + interval '2' hour, + null - interval '2' hour +-- !query analysis +Project [(INTERVAL '2' YEAR + cast(null as interval year)) AS (INTERVAL '2' YEAR + NULL)#x, (INTERVAL '2' YEAR - cast(null as interval year)) AS (INTERVAL '2' YEAR - NULL)#x, (INTERVAL '02' HOUR + cast(null as interval hour)) AS (INTERVAL '02' HOUR + NULL)#x, (INTERVAL '02' HOUR - cast(null as interval hour)) AS (INTERVAL '02' HOUR - NULL)#x, (cast(null as interval year) + INTERVAL '2' YEAR) AS (NULL + INTERVAL '2' YEAR)#x, (cast(null as interval year) - INTERVAL '2' YEAR) AS (NULL - INTERVAL '2' YEAR)#x, (cast(null as interval hour) + INTERVAL '02' HOUR) AS (NULL + INTERVAL '02' HOUR)#x, (cast(null as interval hour) - INTERVAL '02' HOUR) AS (NULL - INTERVAL '02' HOUR)#x] ++- OneRowRelation + + +-- !query +select interval '2' year + '3-3' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "interval '2' year + '3-3'" + } ] +} + + +-- !query +select interval '2' year - '4' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR - 4)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year - '4'" + } ] +} + + +-- !query +select '4 11:11' - interval '4 22:12' day to minute +-- !query analysis +Project [cast(4 11:11 - INTERVAL '4 22:12' DAY TO MINUTE as string) AS 4 11:11 - INTERVAL '4 22:12' DAY TO MINUTE#x] ++- OneRowRelation + + +-- !query +select '4 12:12:12' + interval '4 22:12' day to minute +-- !query analysis +Project [cast(cast(4 12:12:12 as timestamp) + INTERVAL '4 22:12' DAY TO MINUTE as string) AS 4 12:12:12 + INTERVAL '4 22:12' DAY TO MINUTE#x] ++- OneRowRelation + + +-- !query +create temporary view interval_view as select '1' str +-- !query analysis +CreateViewCommand `interval_view`, select '1' str, false, false, LocalTempView, true + +- Project [1 AS str#x] + +- OneRowRelation + + +-- !query +select interval '2' year + str from interval_view +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year + str" + } ] +} + + +-- !query +select interval '2' year - str from interval_view +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR - str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year - str" + } ] +} + + +-- !query +select str - interval '4 22:12' day to minute from interval_view +-- !query analysis +Project [cast(str#x - INTERVAL '4 22:12' DAY TO MINUTE as string) AS str - INTERVAL '4 22:12' DAY TO MINUTE#x] ++- SubqueryAlias interval_view + +- View (`interval_view`, [str#x]) + +- Project [cast(str#x as string) AS str#x] + +- Project [1 AS str#x] + +- OneRowRelation + + +-- !query +select str + interval '4 22:12' day to minute from interval_view +-- !query analysis +Project [cast(cast(str#x as timestamp) + INTERVAL '4 22:12' DAY TO MINUTE as string) AS str + INTERVAL '4 22:12' DAY TO MINUTE#x] ++- SubqueryAlias interval_view + +- View (`interval_view`, [str#x]) + +- Project [cast(str#x as string) AS str#x] + +- Project [1 AS str#x] + +- OneRowRelation + + +-- !query +select interval '2-2' year to month + interval '3' day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"", + "inputType" : "\"INTERVAL YEAR TO MONTH\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '2-2' year to month + interval '3' day" + } ] +} + + +-- !query +select interval '3' day + interval '2-2' year to month +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"", + "inputType" : "\"INTERVAL YEAR TO MONTH\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '3' day + interval '2-2' year to month" + } ] +} + + +-- !query +select interval '2-2' year to month - interval '3' day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"", + "inputType" : "\"INTERVAL YEAR TO MONTH\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + (- INTERVAL '3' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '2-2' year to month - interval '3' day" + } ] +} + + +-- !query +select interval '3' day - interval '2-2' year to month +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"INTERVAL YEAR TO MONTH\"", + "sqlExpr" : "\"(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '3' day - interval '2-2' year to month" + } ] +} + + +-- !query +select 1 - interval '2' second +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"1 + (- INTERVAL '02' SECOND)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "1 - interval '2' second" + } ] +} + + +-- !query +select 1 + interval '2' month +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"INTERVAL MONTH\"", + "sqlExpr" : "\"(1 + INTERVAL '2' MONTH)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "1 + interval '2' month" + } ] +} + + +-- !query +select interval '2' second + 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"1 + INTERVAL '02' SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' second + 1" + } ] +} + + +-- !query +select interval '2' month - 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL MONTH\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(INTERVAL '2' MONTH - 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "interval '2' month - 1" + } ] +} + + +-- !query +select interval '\t interval 1 day' +-- !query analysis +Project [INTERVAL '1' DAY AS INTERVAL '1' DAY#x] ++- OneRowRelation + + +-- !query +select interval 'interval \t 1\tday' +-- !query analysis +Project [INTERVAL '1' DAY AS INTERVAL '1' DAY#x] ++- OneRowRelation + + +-- !query +select interval 'interval\t1\tday' +-- !query analysis +Project [INTERVAL '1' DAY AS INTERVAL '1' DAY#x] ++- OneRowRelation + + +-- !query +select interval '1\t' day +-- !query analysis +Project [INTERVAL '1' DAY AS INTERVAL '1' DAY#x] ++- OneRowRelation + + +-- !query +select interval '1 ' day +-- !query analysis +Project [INTERVAL '1' DAY AS INTERVAL '1' DAY#x] ++- OneRowRelation + + +-- !query +select interval '2-2\t' year to month +-- !query analysis +Project [INTERVAL '2-2' YEAR TO MONTH AS INTERVAL '2-2' YEAR TO MONTH#x] ++- OneRowRelation + + +-- !query +select interval '-\t2-2\t' year to month +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match year-month format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval year to month: -\t2-2\t" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 40, + "fragment" : "'-\\t2-2\\t' year to month" + } ] +} + + +-- !query +select interval '\n0 12:34:46.789\t' day to second +-- !query analysis +Project [INTERVAL '0 12:34:46.789' DAY TO SECOND AS INTERVAL '0 12:34:46.789' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +select interval '\n-\t10\t 12:34:46.789\t' day to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND` when cast to interval day to second: \n-\t10\t 12:34:46.789\t, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 56, + "fragment" : "'\\n-\\t10\\t 12:34:46.789\\t' day to second" + } ] +} + + +-- !query +select interval '中文 interval 1 day' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'中文 interval 1 day'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "interval '中文 interval 1 day'" + } ] +} + + +-- !query +select interval 'interval中文 1 day' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'interval中文 1 day'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "interval 'interval中文 1 day'" + } ] +} + + +-- !query +select interval 'interval 1中文day' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'interval 1中文day'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "interval 'interval 1中文day'" + } ] +} + + +-- !query +select -(a) from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query analysis +Project [-a#x AS (- a)#x] ++- SubqueryAlias t + +- LocalRelation [a#x, b#x] + + +-- !query +select a - b from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query analysis +Project [(a#x - b#x) AS (a - b)#x] ++- SubqueryAlias t + +- LocalRelation [a#x, b#x] + + +-- !query +select b + interval '1 month' from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query analysis +Project [(b#x + INTERVAL '1' MONTH) AS (b + INTERVAL '1' MONTH)#x] ++- SubqueryAlias t + +- LocalRelation [a#x, b#x] + + +-- !query +select a * 1.1 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query analysis +Project [(a#x * 1.1) AS (a * 1.1)#x] ++- SubqueryAlias t + +- LocalRelation [a#x, b#x] + + +-- !query +select a / 0.5 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query analysis +Project [(a#x / 0.5) AS (a / 0.5)#x] ++- SubqueryAlias t + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT + from_csv('1, 1 day', 'a INT, b interval'), + from_csv('1, 1', 'a INT, b interval day'), + to_csv(from_csv('1, 1 day', 'a INT, b interval')), + to_csv(from_csv('1, 1', 'a INT, b interval day')), + to_csv(named_struct('a', interval 32 hour, 'b', interval 70 minute)), + from_csv(to_csv(named_struct('a', interval 32 hour, 'b', interval 70 minute)), 'a interval hour, b interval minute') +-- !query analysis +Project [from_csv(StructField(a,IntegerType,true), StructField(b,CalendarIntervalType,true), 1, 1 day, Some(America/Los_Angeles), None) AS from_csv(1, 1 day)#x, from_csv(StructField(a,IntegerType,true), StructField(b,DayTimeIntervalType(0,0),true), 1, 1, Some(America/Los_Angeles), None) AS from_csv(1, 1)#x, to_csv(from_csv(StructField(a,IntegerType,true), StructField(b,CalendarIntervalType,true), 1, 1 day, Some(America/Los_Angeles), None), Some(America/Los_Angeles)) AS to_csv(from_csv(1, 1 day))#x, to_csv(from_csv(StructField(a,IntegerType,true), StructField(b,DayTimeIntervalType(0,0),true), 1, 1, Some(America/Los_Angeles), None), Some(America/Los_Angeles)) AS to_csv(from_csv(1, 1))#x, to_csv(named_struct(a, INTERVAL '32' HOUR, b, INTERVAL '70' MINUTE), Some(America/Los_Angeles)) AS to_csv(named_struct(a, INTERVAL '32' HOUR, b, INTERVAL '70' MINUTE))#x, from_csv(StructField(a,DayTimeIntervalType(1,1),true), StructField(b,DayTimeIntervalType(2,2),true), to_csv(named_struct(a, INTERVAL '32' HOUR, b, INTERVAL '70' MINUTE), Some(America/Los_Angeles)), Some(America/Los_Angeles), None) AS from_csv(to_csv(named_struct(a, INTERVAL '32' HOUR, b, INTERVAL '70' MINUTE)))#x] ++- OneRowRelation + + +-- !query +SELECT + from_json('{"a":"1 days"}', 'a interval'), + from_csv('1, 1', 'a INT, b interval year'), + to_json(from_json('{"a":"1 days"}', 'a interval')), + to_csv(from_csv('1, 1', 'a INT, b interval year')), + to_csv(named_struct('a', interval 32 year, 'b', interval 10 month)), + from_csv(to_csv(named_struct('a', interval 32 year, 'b', interval 10 month)), 'a interval year, b interval month') +-- !query analysis +Project [from_json(StructField(a,CalendarIntervalType,true), {"a":"1 days"}, Some(America/Los_Angeles)) AS from_json({"a":"1 days"})#x, from_csv(StructField(a,IntegerType,true), StructField(b,YearMonthIntervalType(0,0),true), 1, 1, Some(America/Los_Angeles), None) AS from_csv(1, 1)#x, to_json(from_json(StructField(a,CalendarIntervalType,true), {"a":"1 days"}, Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS to_json(from_json({"a":"1 days"}))#x, to_csv(from_csv(StructField(a,IntegerType,true), StructField(b,YearMonthIntervalType(0,0),true), 1, 1, Some(America/Los_Angeles), None), Some(America/Los_Angeles)) AS to_csv(from_csv(1, 1))#x, to_csv(named_struct(a, INTERVAL '32' YEAR, b, INTERVAL '10' MONTH), Some(America/Los_Angeles)) AS to_csv(named_struct(a, INTERVAL '32' YEAR, b, INTERVAL '10' MONTH))#x, from_csv(StructField(a,YearMonthIntervalType(0,0),true), StructField(b,YearMonthIntervalType(1,1),true), to_csv(named_struct(a, INTERVAL '32' YEAR, b, INTERVAL '10' MONTH), Some(America/Los_Angeles)), Some(America/Los_Angeles), None) AS from_csv(to_csv(named_struct(a, INTERVAL '32' YEAR, b, INTERVAL '10' MONTH)))#x] ++- OneRowRelation + + +-- !query +SELECT + from_json('{"a":"1"}', 'a interval day'), + to_json(from_json('{"a":"1"}', 'a interval day')), + to_json(map('a', interval 100 day 130 minute)), + from_json(to_json(map('a', interval 100 day 130 minute)), 'a interval day to minute') +-- !query analysis +Project [from_json(StructField(a,DayTimeIntervalType(0,0),true), {"a":"1"}, Some(America/Los_Angeles)) AS from_json({"a":"1"})#x, to_json(from_json(StructField(a,DayTimeIntervalType(0,0),true), {"a":"1"}, Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS to_json(from_json({"a":"1"}))#x, to_json(map(a, INTERVAL '100 02:10' DAY TO MINUTE), Some(America/Los_Angeles)) AS to_json(map(a, INTERVAL '100 02:10' DAY TO MINUTE))#x, from_json(StructField(a,DayTimeIntervalType(0,2),true), to_json(map(a, INTERVAL '100 02:10' DAY TO MINUTE), Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS from_json(to_json(map(a, INTERVAL '100 02:10' DAY TO MINUTE)))#x] ++- OneRowRelation + + +-- !query +SELECT + from_json('{"a":"1"}', 'a interval year'), + to_json(from_json('{"a":"1"}', 'a interval year')), + to_json(map('a', interval 32 year 10 month)), + from_json(to_json(map('a', interval 32 year 10 month)), 'a interval year to month') +-- !query analysis +Project [from_json(StructField(a,YearMonthIntervalType(0,0),true), {"a":"1"}, Some(America/Los_Angeles)) AS from_json({"a":"1"})#x, to_json(from_json(StructField(a,YearMonthIntervalType(0,0),true), {"a":"1"}, Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS to_json(from_json({"a":"1"}))#x, to_json(map(a, INTERVAL '32-10' YEAR TO MONTH), Some(America/Los_Angeles)) AS to_json(map(a, INTERVAL '32-10' YEAR TO MONTH))#x, from_json(StructField(a,YearMonthIntervalType(0,1),true), to_json(map(a, INTERVAL '32-10' YEAR TO MONTH), Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS from_json(to_json(map(a, INTERVAL '32-10' YEAR TO MONTH)))#x] ++- OneRowRelation + + +-- !query +select interval '+' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'+'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "interval '+'" + } ] +} + + +-- !query +select interval '+.' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'+.'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "interval '+.'" + } ] +} + + +-- !query +select interval '1' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "interval '1'" + } ] +} + + +-- !query +select interval '1.2' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1.2'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval '1.2'" + } ] +} + + +-- !query +select interval '- 2' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'- 2'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval '- 2'" + } ] +} + + +-- !query +select interval '1 day -' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1 day -'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "interval '1 day -'" + } ] +} + + +-- !query +select interval '1 day 1' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1 day 1'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "interval '1 day 1'" + } ] +} + + +-- !query +select interval '1 day 2' day +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0026", + "messageParameters" : { + "value" : "1 day 2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 29, + "fragment" : "'1 day 2' day" + } ] +} + + +-- !query +select interval 'interval 1' day +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0026", + "messageParameters" : { + "value" : "interval 1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 32, + "fragment" : "'interval 1' day" + } ] +} + + +-- !query +select interval '-\t 1' day +-- !query analysis +Project [INTERVAL '-1' DAY AS INTERVAL '-1' DAY#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / 2 +-- !query analysis +Project [(INTERVAL '-178956970-8' YEAR TO MONTH / 2) AS (INTERVAL '-178956970-8' YEAR TO MONTH / 2)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / 5 +-- !query analysis +Project [(INTERVAL '-178956970-8' YEAR TO MONTH / 5) AS (INTERVAL '-178956970-8' YEAR TO MONTH / 5)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1 +-- !query analysis +Project [(INTERVAL '-178956970-8' YEAR TO MONTH / -1) AS (INTERVAL '-178956970-8' YEAR TO MONTH / -1)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1L +-- !query analysis +Project [(INTERVAL '-178956970-8' YEAR TO MONTH / -1) AS (INTERVAL '-178956970-8' YEAR TO MONTH / -1)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1.0 +-- !query analysis +Project [(INTERVAL '-178956970-8' YEAR TO MONTH / -1.0) AS (INTERVAL '-178956970-8' YEAR TO MONTH / -1.0)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1.0D +-- !query analysis +Project [(INTERVAL '-178956970-8' YEAR TO MONTH / -1.0) AS (INTERVAL '-178956970-8' YEAR TO MONTH / -1.0)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / 2 +-- !query analysis +Project [(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / 2) AS (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / 2)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / 5 +-- !query analysis +Project [(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / 5) AS (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / 5)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1 +-- !query analysis +Project [(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1) AS (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L +-- !query analysis +Project [(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1) AS (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1.0 +-- !query analysis +Project [(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1.0) AS (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1.0)#x] ++- OneRowRelation + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1.0D +-- !query analysis +Project [(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1.0) AS (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / -1.0)#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '106751991 04' DAY TO HOUR +-- !query analysis +Project [INTERVAL '106751991 04' DAY TO HOUR AS INTERVAL '106751991 04' DAY TO HOUR#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '106751991 04:00' DAY TO MINUTE +-- !query analysis +Project [INTERVAL '106751991 04:00' DAY TO MINUTE AS INTERVAL '106751991 04:00' DAY TO MINUTE#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '106751991 04:00:54.775807' DAY TO SECOND +-- !query analysis +Project [INTERVAL '106751991 04:00:54.775807' DAY TO SECOND AS INTERVAL '106751991 04:00:54.775807' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '2562047788:00' HOUR TO MINUTE +-- !query analysis +Project [INTERVAL '2562047788:00' HOUR TO MINUTE AS INTERVAL '2562047788:00' HOUR TO MINUTE#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '2562047788:00:54.775807' HOUR TO SECOND +-- !query analysis +Project [INTERVAL '2562047788:00:54.775807' HOUR TO SECOND AS INTERVAL '2562047788:00:54.775807' HOUR TO SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '153722867280:54.775807' MINUTE TO SECOND +-- !query analysis +Project [INTERVAL '153722867280:54.775807' MINUTE TO SECOND AS INTERVAL '153722867280:54.775807' MINUTE TO SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-106751991 04' DAY TO HOUR +-- !query analysis +Project [INTERVAL '-106751991 04' DAY TO HOUR AS INTERVAL '-106751991 04' DAY TO HOUR#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-106751991 04:00' DAY TO MINUTE +-- !query analysis +Project [INTERVAL '-106751991 04:00' DAY TO MINUTE AS INTERVAL '-106751991 04:00' DAY TO MINUTE#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND +-- !query analysis +Project [INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND AS INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-2562047788:00' HOUR TO MINUTE +-- !query analysis +Project [INTERVAL '-2562047788:00' HOUR TO MINUTE AS INTERVAL '-2562047788:00' HOUR TO MINUTE#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-2562047788:00:54.775808' HOUR TO SECOND +-- !query analysis +Project [INTERVAL '-2562047788:00:54.775808' HOUR TO SECOND AS INTERVAL '-2562047788:00:54.775808' HOUR TO SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-153722867280:54.775808' MINUTE TO SECOND +-- !query analysis +Project [INTERVAL '-153722867280:54.775808' MINUTE TO SECOND AS INTERVAL '-153722867280:54.775808' MINUTE TO SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '106751992 04' DAY TO HOUR +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: day 106751992 outside range [0, 106751991]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 42, + "fragment" : "'106751992 04' DAY TO HOUR" + } ] +} + + +-- !query +SELECT INTERVAL '-106751992 04' DAY TO HOUR +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: day 106751992 outside range [0, 106751991]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 43, + "fragment" : "'-106751992 04' DAY TO HOUR" + } ] +} + + +-- !query +SELECT INTERVAL '2562047789:00' HOUR TO MINUTE +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 46, + "fragment" : "'2562047789:00' HOUR TO MINUTE" + } ] +} + + +-- !query +SELECT INTERVAL '-2562047789:00' HOUR TO MINUTE +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 47, + "fragment" : "'-2562047789:00' HOUR TO MINUTE" + } ] +} + + +-- !query +SELECT INTERVAL '153722867281:54.775808' MINUTE TO SECOND +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 57, + "fragment" : "'153722867281:54.775808' MINUTE TO SECOND" + } ] +} + + +-- !query +SELECT INTERVAL '-153722867281:54.775808' MINUTE TO SECOND +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 58, + "fragment" : "'-153722867281:54.775808' MINUTE TO SECOND" + } ] +} + + +-- !query +SELECT INTERVAL '178956970' YEAR +-- !query analysis +Project [INTERVAL '178956970' YEAR AS INTERVAL '178956970' YEAR#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-178956970' YEAR +-- !query analysis +Project [INTERVAL '-178956970' YEAR AS INTERVAL '-178956970' YEAR#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '2147483647' MONTH +-- !query analysis +Project [INTERVAL '2147483647' MONTH AS INTERVAL '2147483647' MONTH#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-2147483647' MONTH +-- !query analysis +Project [INTERVAL '-2147483647' MONTH AS INTERVAL '-2147483647' MONTH#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '106751991' DAY +-- !query analysis +Project [INTERVAL '106751991' DAY AS INTERVAL '106751991' DAY#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-106751991' DAY +-- !query analysis +Project [INTERVAL '-106751991' DAY AS INTERVAL '-106751991' DAY#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '2562047788' HOUR +-- !query analysis +Project [INTERVAL '2562047788' HOUR AS INTERVAL '2562047788' HOUR#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-2562047788' HOUR +-- !query analysis +Project [INTERVAL '-2562047788' HOUR AS INTERVAL '-2562047788' HOUR#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '153722867280' MINUTE +-- !query analysis +Project [INTERVAL '153722867280' MINUTE AS INTERVAL '153722867280' MINUTE#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-153722867280' MINUTE +-- !query analysis +Project [INTERVAL '-153722867280' MINUTE AS INTERVAL '-153722867280' MINUTE#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '54.775807' SECOND +-- !query analysis +Project [INTERVAL '54.775807' SECOND AS INTERVAL '54.775807' SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-54.775807' SECOND +-- !query analysis +Project [INTERVAL '-54.775807' SECOND AS INTERVAL '-54.775807' SECOND#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '1' DAY > INTERVAL '1' HOUR +-- !query analysis +Project [(cast(INTERVAL '1' DAY as interval day to hour) > cast(INTERVAL '01' HOUR as interval day to hour)) AS (INTERVAL '1' DAY > INTERVAL '01' HOUR)#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '1 02' DAY TO HOUR = INTERVAL '02:10:55' HOUR TO SECOND +-- !query analysis +Project [(cast(INTERVAL '1 02' DAY TO HOUR as interval day to second) = cast(INTERVAL '02:10:55' HOUR TO SECOND as interval day to second)) AS (INTERVAL '1 02' DAY TO HOUR = INTERVAL '02:10:55' HOUR TO SECOND)#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '1' YEAR < INTERVAL '1' MONTH +-- !query analysis +Project [(cast(INTERVAL '1' YEAR as interval year to month) < cast(INTERVAL '1' MONTH as interval year to month)) AS (INTERVAL '1' YEAR < INTERVAL '1' MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL '-1-1' YEAR TO MONTH = INTERVAL '-13' MONTH +-- !query analysis +Project [(INTERVAL '-1-1' YEAR TO MONTH = cast(INTERVAL '-13' MONTH as interval year to month)) AS (INTERVAL '-1-1' YEAR TO MONTH = INTERVAL '-13' MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL MONTH\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(INTERVAL '1' MONTH > INTERVAL '20' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "INTERVAL 1 MONTH > INTERVAL 20 DAYS" + } ] +} + + +-- !query +SELECT INTERVAL '1' DAY < '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY < 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "INTERVAL '1' DAY < '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' DAY = '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY = 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "INTERVAL '1' DAY = '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' DAY > '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY > 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "INTERVAL '1' DAY > '1'" + } ] +} + + +-- !query +SELECT '1' < INTERVAL '1' DAY +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 < INTERVAL '1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' < INTERVAL '1' DAY" + } ] +} + + +-- !query +SELECT '1' = INTERVAL '1' DAY +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 = INTERVAL '1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' = INTERVAL '1' DAY" + } ] +} + + +-- !query +SELECT '1' > INTERVAL '1' DAY +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 > INTERVAL '1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' > INTERVAL '1' DAY" + } ] +} + + +-- !query +SELECT INTERVAL '1' YEAR < '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR < 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "INTERVAL '1' YEAR < '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' YEAR = '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR = 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "INTERVAL '1' YEAR = '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' YEAR > '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR > 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "INTERVAL '1' YEAR > '1'" + } ] +} + + +-- !query +SELECT '1' < INTERVAL '1' YEAR +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 < INTERVAL '1' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'1' < INTERVAL '1' YEAR" + } ] +} + + +-- !query +SELECT '1' = INTERVAL '1' YEAR +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 = INTERVAL '1' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'1' = INTERVAL '1' YEAR" + } ] +} + + +-- !query +SELECT '1' > INTERVAL '1' YEAR +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 > INTERVAL '1' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'1' > INTERVAL '1' YEAR" + } ] +} + + +-- !query +SELECT array(INTERVAL '1' YEAR, INTERVAL '1' MONTH) +-- !query analysis +Project [array(cast(INTERVAL '1' YEAR as interval year to month), cast(INTERVAL '1' MONTH as interval year to month)) AS array(INTERVAL '1' YEAR, INTERVAL '1' MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT array(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE) +-- !query analysis +Project [array(cast(INTERVAL '1' DAY as interval day to minute), cast(INTERVAL '01:01' HOUR TO MINUTE as interval day to minute)) AS array(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE)#x] ++- OneRowRelation + + +-- !query +SELECT array(INTERVAL 1 MONTH, INTERVAL 20 DAYS) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"INTERVAL MONTH\" or \"INTERVAL DAY\")", + "functionName" : "`array`", + "sqlExpr" : "\"array(INTERVAL '1' MONTH, INTERVAL '20' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "array(INTERVAL 1 MONTH, INTERVAL 20 DAYS)" + } ] +} + + +-- !query +SELECT coalesce(INTERVAL '1' YEAR, INTERVAL '1' MONTH) +-- !query analysis +Project [coalesce(cast(INTERVAL '1' YEAR as interval year to month), cast(INTERVAL '1' MONTH as interval year to month)) AS coalesce(INTERVAL '1' YEAR, INTERVAL '1' MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT coalesce(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE) +-- !query analysis +Project [coalesce(cast(INTERVAL '1' DAY as interval day to minute), cast(INTERVAL '01:01' HOUR TO MINUTE as interval day to minute)) AS coalesce(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE)#x] ++- OneRowRelation + + +-- !query +SELECT coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"INTERVAL MONTH\" or \"INTERVAL DAY\")", + "functionName" : "`coalesce`", + "sqlExpr" : "\"coalesce(INTERVAL '1' MONTH, INTERVAL '20' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS)" + } ] +} + + +-- !query +SELECT abs(INTERVAL '-10' YEAR) +-- !query analysis +Project [abs(INTERVAL '-10' YEAR) AS abs(INTERVAL '-10' YEAR)#x] ++- OneRowRelation + + +-- !query +SELECT abs(INTERVAL -'1 02:03:04.123' DAY TO SECOND) +-- !query analysis +Project [abs(INTERVAL '-1 02:03:04.123' DAY TO SECOND) AS abs(INTERVAL '-1 02:03:04.123' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +SELECT div(INTERVAL '1-1' YEAR TO MONTH, INTERVAL '1' YEAR) +-- !query analysis +Project [(INTERVAL '1-1' YEAR TO MONTH div cast(INTERVAL '1' YEAR as interval year to month)) AS (INTERVAL '1-1' YEAR TO MONTH div INTERVAL '1' YEAR)#xL] ++- OneRowRelation + + +-- !query +SELECT div(INTERVAL '1-1' YEAR TO MONTH, INTERVAL '-1' MONTH) +-- !query analysis +Project [(INTERVAL '1-1' YEAR TO MONTH div cast(INTERVAL '-1' MONTH as interval year to month)) AS (INTERVAL '1-1' YEAR TO MONTH div INTERVAL '-1' MONTH)#xL] ++- OneRowRelation + + +-- !query +SELECT div(INTERVAL '1 06' DAY TO HOUR, INTERVAL '1' DAY) +-- !query analysis +Project [(INTERVAL '1 06' DAY TO HOUR div cast(INTERVAL '1' DAY as interval day to hour)) AS (INTERVAL '1 06' DAY TO HOUR div INTERVAL '1' DAY)#xL] ++- OneRowRelation + + +-- !query +SELECT div(INTERVAL '1 06' DAY TO HOUR, INTERVAL '-1' HOUR) +-- !query analysis +Project [(INTERVAL '1 06' DAY TO HOUR div cast(INTERVAL '-01' HOUR as interval day to hour)) AS (INTERVAL '1 06' DAY TO HOUR div INTERVAL '-01' HOUR)#xL] ++- OneRowRelation + + +-- !query +SELECT div(INTERVAL '1' MONTH, INTERVAL '-1' DAY) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL MONTH\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(INTERVAL '1' MONTH div INTERVAL '-1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "div(INTERVAL '1' MONTH, INTERVAL '-1' DAY)" + } ] +} + + +-- !query +SELECT signum(INTERVAL '-10' YEAR) +-- !query analysis +Project [SIGNUM(INTERVAL '-10' YEAR) AS SIGNUM(INTERVAL '-10' YEAR)#x] ++- OneRowRelation + + +-- !query +SELECT signum(INTERVAL '10' MONTH) +-- !query analysis +Project [SIGNUM(INTERVAL '10' MONTH) AS SIGNUM(INTERVAL '10' MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT signum(INTERVAL '0-0' YEAR TO MONTH) +-- !query analysis +Project [SIGNUM(INTERVAL '0-0' YEAR TO MONTH) AS SIGNUM(INTERVAL '0-0' YEAR TO MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT signum(INTERVAL '-10' DAY) +-- !query analysis +Project [SIGNUM(INTERVAL '-10' DAY) AS SIGNUM(INTERVAL '-10' DAY)#x] ++- OneRowRelation + + +-- !query +SELECT signum(INTERVAL '10' HOUR) +-- !query analysis +Project [SIGNUM(INTERVAL '10' HOUR) AS SIGNUM(INTERVAL '10' HOUR)#x] ++- OneRowRelation + + +-- !query +SELECT signum(INTERVAL '0 0:0:0' DAY TO SECOND) +-- !query analysis +Project [SIGNUM(INTERVAL '0 00:00:00' DAY TO SECOND) AS SIGNUM(INTERVAL '0 00:00:00' DAY TO SECOND)#x] ++- OneRowRelation + + +-- !query +SELECT width_bucket(INTERVAL '0' YEAR, INTERVAL '0' YEAR, INTERVAL '10' YEAR, 10) +-- !query analysis +Project [width_bucket(INTERVAL '0' YEAR, INTERVAL '0' YEAR, INTERVAL '10' YEAR, cast(10 as bigint)) AS width_bucket(INTERVAL '0' YEAR, INTERVAL '0' YEAR, INTERVAL '10' YEAR, 10)#xL] ++- OneRowRelation + + +-- !query +SELECT width_bucket(INTERVAL '-1' YEAR, INTERVAL -'1-2' YEAR TO MONTH, INTERVAL '1-2' YEAR TO MONTH, 10) +-- !query analysis +Project [width_bucket(INTERVAL '-1' YEAR, INTERVAL '-1-2' YEAR TO MONTH, INTERVAL '1-2' YEAR TO MONTH, cast(10 as bigint)) AS width_bucket(INTERVAL '-1' YEAR, INTERVAL '-1-2' YEAR TO MONTH, INTERVAL '1-2' YEAR TO MONTH, 10)#xL] ++- OneRowRelation + + +-- !query +SELECT width_bucket(INTERVAL '0' DAY, INTERVAL '0' DAY, INTERVAL '10' DAY, 10) +-- !query analysis +Project [width_bucket(INTERVAL '0' DAY, INTERVAL '0' DAY, INTERVAL '10' DAY, cast(10 as bigint)) AS width_bucket(INTERVAL '0' DAY, INTERVAL '0' DAY, INTERVAL '10' DAY, 10)#xL] ++- OneRowRelation + + +-- !query +SELECT width_bucket(INTERVAL '-59' MINUTE, INTERVAL -'1 01' DAY TO HOUR, INTERVAL '1 2:3:4.001' DAY TO SECOND, 10) +-- !query analysis +Project [width_bucket(INTERVAL '-59' MINUTE, INTERVAL '-1 01' DAY TO HOUR, INTERVAL '1 02:03:04.001' DAY TO SECOND, cast(10 as bigint)) AS width_bucket(INTERVAL '-59' MINUTE, INTERVAL '-1 01' DAY TO HOUR, INTERVAL '1 02:03:04.001' DAY TO SECOND, 10)#xL] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/join-empty-relation.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/join-empty-relation.sql.out new file mode 100644 index 0000000000000..a4fae64952048 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/join-empty-relation.sql.out @@ -0,0 +1,570 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query analysis +CreateViewCommand `t1`, SELECT * FROM VALUES (1) AS GROUPING(a), false, false, LocalTempView, true + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query analysis +CreateViewCommand `t2`, SELECT * FROM VALUES (1) AS GROUPING(a), false, false, LocalTempView, true + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +CREATE TEMPORARY VIEW empty_table as SELECT a FROM t2 WHERE false +-- !query analysis +CreateViewCommand `empty_table`, SELECT a FROM t2 WHERE false, false, false, LocalTempView, true + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM t1 INNER JOIN empty_table +-- !query analysis +Project [a#x, a#x] ++- Join Inner + :- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM t1 CROSS JOIN empty_table +-- !query analysis +Project [a#x, a#x] ++- Join Cross + :- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM t1 LEFT OUTER JOIN empty_table +-- !query analysis +Project [a#x, a#x] ++- Join LeftOuter + :- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM t1 RIGHT OUTER JOIN empty_table +-- !query analysis +Project [a#x, a#x] ++- Join RightOuter + :- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM t1 FULL OUTER JOIN empty_table +-- !query analysis +Project [a#x, a#x] ++- Join FullOuter + :- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM t1 LEFT SEMI JOIN empty_table +-- !query analysis +Project [a#x] ++- Join LeftSemi + :- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM t1 LEFT ANTI JOIN empty_table +-- !query analysis +Project [a#x] ++- Join LeftAnti + :- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM empty_table INNER JOIN t1 +-- !query analysis +Project [a#x, a#x] ++- Join Inner + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias t1 + +- View (`t1`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM empty_table CROSS JOIN t1 +-- !query analysis +Project [a#x, a#x] ++- Join Cross + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias t1 + +- View (`t1`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM empty_table LEFT OUTER JOIN t1 +-- !query analysis +Project [a#x, a#x] ++- Join LeftOuter + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias t1 + +- View (`t1`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM empty_table RIGHT OUTER JOIN t1 +-- !query analysis +Project [a#x, a#x] ++- Join RightOuter + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias t1 + +- View (`t1`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM empty_table FULL OUTER JOIN t1 +-- !query analysis +Project [a#x, a#x] ++- Join FullOuter + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias t1 + +- View (`t1`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM empty_table LEFT SEMI JOIN t1 +-- !query analysis +Project [a#x] ++- Join LeftSemi + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias t1 + +- View (`t1`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM empty_table LEFT ANTI JOIN t1 +-- !query analysis +Project [a#x] ++- Join LeftAnti + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias t1 + +- View (`t1`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM empty_table INNER JOIN empty_table +-- !query analysis +Project [a#x, a#x] ++- Join Inner + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM empty_table CROSS JOIN empty_table +-- !query analysis +Project [a#x, a#x] ++- Join Cross + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM empty_table LEFT OUTER JOIN empty_table +-- !query analysis +Project [a#x, a#x] ++- Join LeftOuter + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM empty_table RIGHT OUTER JOIN empty_table +-- !query analysis +Project [a#x, a#x] ++- Join RightOuter + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM empty_table FULL OUTER JOIN empty_table +-- !query analysis +Project [a#x, a#x] ++- Join FullOuter + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM empty_table LEFT SEMI JOIN empty_table +-- !query analysis +Project [a#x] ++- Join LeftSemi + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT * FROM empty_table LEFT ANTI JOIN empty_table +-- !query analysis +Project [a#x] ++- Join LeftAnti + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out new file mode 100644 index 0000000000000..09218baebba0c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out @@ -0,0 +1,2812 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2) +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`t1`, [(c1,None), (c2,None)], VALUES (0, 1), (1, 2), false, false, PersistedView, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE VIEW t2(c1, c2) AS VALUES (0, 2), (0, 3) +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`t2`, [(c1,None), (c2,None)], VALUES (0, 2), (0, 3), false, false, PersistedView, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE VIEW t3(c1, c2) AS VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4)) +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`t3`, [(c1,None), (c2,None)], VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4)), false, false, PersistedView, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE VIEW t4(c1, c2) AS VALUES (0, 1), (0, 2), (1, 1), (1, 3) +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`t4`, [(c1,None), (c2,None)], VALUES (0, 1), (0, 2), (1, 1), (1, 3), false, false, PersistedView, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c1) +-- !query analysis +Project [c1#x, c2#x, c1#x] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [outer(c1#x) AS c1#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c1 FROM t2) +-- !query analysis +Project [c1#x, c2#x, c1#x] ++- LateralJoin lateral-subquery#x [], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [c1#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.c1 FROM t2) +-- !query analysis +Project [c1#x, c2#x, c1#x] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [outer(c1#x) AS c1#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.c1 + t2.c1 FROM t2) +-- !query analysis +Project [c1#x, c2#x, (outer(spark_catalog.default.t1.c1) + c1)#x] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [(outer(c1#x) + c1#x) AS (outer(spark_catalog.default.t1.c1) + c1)#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT *) +-- !query analysis +Project [c1#x, c2#x] ++- LateralJoin lateral-subquery#x [], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM t2) +-- !query analysis +Project [c1#x, c2#x, c1#x, c2#x] ++- LateralJoin lateral-subquery#x [], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [c1#x, c2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.*) +-- !query analysis +Project [c1#x, c2#x, c1#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [outer(c1#x) AS c1#x, outer(c2#x) AS c2#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.*, t2.* FROM t2) +-- !query analysis +Project [c1#x, c2#x, c1#x, c2#x, c1#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [outer(c1#x) AS c1#x, outer(c2#x) AS c2#x, c1#x, c2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.* FROM t2 AS t1) +-- !query analysis +Project [c1#x, c2#x, c1#x, c2#x] ++- LateralJoin lateral-subquery#x [], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [c1#x, c2#x] + : +- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.*, t2.* FROM t2, LATERAL (SELECT t1.*, t2.*, t3.* FROM t2 AS t3)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1051", + "messageParameters" : { + "columns" : "c1, c2", + "targetString" : "t1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 74, + "fragment" : "t1.*" + } ] +} + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3 +-- !query analysis +Project [c1#x, c2#x, c3#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner, (c2#x = c3#x) + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [(outer(c1#x) + outer(c2#x)) AS c3#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 LEFT JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3 +-- !query analysis +Project [c1#x, c2#x, c3#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], LeftOuter, (c2#x = c3#x) + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [(outer(c1#x) + outer(c2#x)) AS c3#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 CROSS JOIN LATERAL (SELECT c1 + c2 AS c3) +-- !query analysis +Project [c1#x, c2#x, c3#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Cross + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [(outer(c1#x) + outer(c2#x)) AS c3#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INCOMPATIBLE_JOIN_TYPES", + "sqlState" : "42613", + "messageParameters" : { + "joinType1" : "LATERAL", + "joinType2" : "NATURAL" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 18, + "stopIndex" : 60, + "fragment" : "NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2)" + } ] +} + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.LATERAL_JOIN_USING", + "sqlState" : "0A000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 18, + "stopIndex" : 63, + "fragment" : "JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2)" + } ] +} + + +-- !query +SELECT * FROM LATERAL (SELECT * FROM t1) +-- !query analysis +Project [c1#x, c2#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM t2) +-- !query analysis +Project [c1#x, c2#x, c1#x, c2#x] ++- LateralJoin lateral-subquery#x [], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [c1#x, c2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM LATERAL (SELECT * FROM t1), LATERAL (SELECT * FROM t2) +-- !query analysis +Project [c1#x, c2#x, c1#x, c2#x] ++- LateralJoin lateral-subquery#x [], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [c1#x, c2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM LATERAL (SELECT * FROM t1) JOIN LATERAL (SELECT * FROM t2) +-- !query analysis +Project [c1#x, c2#x, c1#x, c2#x] ++- LateralJoin lateral-subquery#x [], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [c1#x, c2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [c1#x, c2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT a, b FROM t1, LATERAL (SELECT c1, c2) s(a, b) +-- !query analysis +Project [a#x, b#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias s + : +- Project [c1#x AS a#x, c2#x AS b#x] + : +- Project [outer(c1#x), outer(c2#x)] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM (SELECT 1 AS c1, 2 AS c2), LATERAL (SELECT c1, c2) +-- !query analysis +Project [c1#x, c2#x, c1#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [outer(c1#x) AS c1#x, outer(c2#x) AS c2#x] + : +- OneRowRelation + +- SubqueryAlias __auto_generated_subquery_name + +- Project [1 AS c1#x, 2 AS c2#x] + +- OneRowRelation + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c2 FROM t2 WHERE t1.c1 = t2.c1) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [c2#x] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c2 FROM t2 WHERE t1.c2 < t2.c2) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [c2#x] + : +- Filter (outer(c2#x) < c2#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT max(c2) AS m FROM t2 WHERE t1.c2 < t2.c2) +-- !query analysis +Project [c1#x, c2#x, m#x] ++- LateralJoin lateral-subquery#x [c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [max(c2#x) AS m#x] + : +- Filter (outer(c2#x) < c2#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN t2 JOIN LATERAL (SELECT t1.c2 + t2.c2) +-- !query analysis +Project [c1#x, c2#x, c1#x, c2#x, (outer(spark_catalog.default.t1.c2) + outer(spark_catalog.default.t2.c2))#x] ++- LateralJoin lateral-subquery#x [c2#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [(outer(c2#x) + outer(c2#x)) AS (outer(spark_catalog.default.t1.c2) + outer(spark_catalog.default.t2.c2))#x] + : +- OneRowRelation + +- Join Inner + :- SubqueryAlias spark_catalog.default.t1 + : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t2 + +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT t1.c1 AS a, t2.c1 AS b) s JOIN t2 ON s.b = t2.c1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t2`.`c1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 55, + "fragment" : "t2.c1" + } ] +} + + +-- !query +SELECT x FROM VALUES (0) t(x) JOIN LATERAL (SELECT x + rand(0) AS y) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT x FROM (SELECT SUM(c1) AS x FROM t1), LATERAL (SELECT x + rand(0) AS y) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c1 + c2 + rand(0) AS c3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_DETERMINISTIC_LATERAL_SUBQUERIES", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "LateralJoin lateral-subquery#x [c1#x && c2#x], Inner\n: +- SubqueryAlias __auto_generated_subquery_name\n: +- Project [(cast((outer(c1#x) + outer(c2#x)) as double) + rand(0)) AS c3#x]\n: +- OneRowRelation\n+- SubqueryAlias spark_catalog.default.t1\n +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 58, + "fragment" : "FROM t1, LATERAL (SELECT c1 + c2 + rand(0) AS c3)" + } ] +} + + +-- !query +SELECT * FROM t1, LATERAL (SELECT rand(0) FROM t2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_DETERMINISTIC_LATERAL_SUBQUERIES", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "LateralJoin lateral-subquery#x [], Inner\n: +- SubqueryAlias __auto_generated_subquery_name\n: +- Project [rand(0) AS rand(0)#x]\n: +- SubqueryAlias spark_catalog.default.t2\n: +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x])\n: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n: +- LocalRelation [col1#x, col2#x]\n+- SubqueryAlias spark_catalog.default.t1\n +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 50, + "fragment" : "FROM t1, LATERAL (SELECT rand(0) FROM t2)" + } ] +} + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT * FROM t2) s ON t1.c1 + rand(0) = s.c1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.LATERAL_JOIN_CONDITION_NON_DETERMINISTIC", + "sqlState" : "0A000", + "messageParameters" : { + "condition" : "((CAST(spark_catalog.default.t1.c1 AS DOUBLE) + rand(0)) = CAST(s.c1 AS DOUBLE))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 55, + "stopIndex" : 76, + "fragment" : "t1.c1 + rand(0) = s.c1" + } ] +} + + +-- !query +SELECT * FROM t1, +LATERAL (SELECT c1 + c2 AS a), +LATERAL (SELECT c1 - c2 AS b), +LATERAL (SELECT a * b AS c) +-- !query analysis +Project [c1#x, c2#x, a#x, b#x, c#x] ++- LateralJoin lateral-subquery#x [a#x && b#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [(outer(a#x) * outer(b#x)) AS c#x] + : +- OneRowRelation + +- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [(outer(c1#x) - outer(c2#x)) AS b#x] + : +- OneRowRelation + +- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [(outer(c1#x) + outer(c2#x)) AS a#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 +LEFT OUTER JOIN LATERAL (SELECT c2 FROM t2 WHERE t1.c1 = t2.c1) s +LEFT OUTER JOIN t1 t3 ON s.c2 = t3.c2 +-- !query analysis +Project [c1#x, c2#x, c2#x, c1#x, c2#x] ++- Join LeftOuter, (c2#x = c2#x) + :- LateralJoin lateral-subquery#x [c1#x], LeftOuter + : : +- SubqueryAlias s + : : +- Project [c2#x] + : : +- Filter (outer(c1#x) = c1#x) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias spark_catalog.default.t1 + : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias t3 + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT c1)) +-- !query analysis +Project [c1#x, c2#x, c1#x, c2#x, c1#x] ++- LateralJoin lateral-subquery#x [], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [c1#x, c2#x, c1#x] + : +- LateralJoin lateral-subquery#x [c1#x], Inner + : : +- SubqueryAlias __auto_generated_subquery_name + : : +- Project [outer(c1#x) AS c1#x] + : : +- OneRowRelation + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1 + 1 AS c1), LATERAL (SELECT c1)) +-- !query analysis +Project [c1#x, c2#x, c1#x, c1#x] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [c1#x, c1#x] + : +- LateralJoin lateral-subquery#x [c1#x], Inner + : : +- SubqueryAlias __auto_generated_subquery_name + : : +- Project [outer(c1#x) AS c1#x] + : : +- OneRowRelation + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [(outer(c1#x) + 1) AS c1#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL ( + SELECT * FROM (SELECT c1, MIN(c2) m FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c1) s, + LATERAL (SELECT m WHERE m > c1) +) +-- !query analysis +Project [c1#x, c2#x, c1#x, m#x, m#x] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [c1#x, m#x, m#x] + : +- LateralJoin lateral-subquery#x [m#x && m#x && c1#x], Inner + : : +- SubqueryAlias __auto_generated_subquery_name + : : +- Project [outer(m#x) AS m#x] + : : +- Filter (outer(m#x) > outer(c1#x)) + : : +- OneRowRelation + : +- SubqueryAlias s + : +- Aggregate [c1#x], [c1#x, min(c2#x) AS m#x] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.c1 + t2.c1)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1`.`c1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 66, + "fragment" : "t1.c1" + } ] +} + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1), LATERAL (SELECT c2)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`c2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 72, + "fragment" : "c2" + } ] +} + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c2, (SELECT MIN(c2) FROM t2)) +-- !query analysis +Project [c1#x, c2#x, c2#x, scalarsubquery()#x] ++- LateralJoin lateral-subquery#x [c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [outer(c2#x) AS c2#x, scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Aggregate [min(c2#x) AS min(c2)#x] + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT (SELECT SUM(c2) FROM t2 WHERE c1 = a) FROM (SELECT c1 AS a)) +-- !query analysis +Project [c1#x, c2#x, scalarsubquery(a)#xL] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [scalar-subquery#x [a#x] AS scalarsubquery(a)#xL] + : : +- Aggregate [sum(c2#x) AS sum(c2)#xL] + : : +- Filter (c1#x = outer(a#x)) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [outer(c1#x) AS a#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c1, (SELECT SUM(c2) FROM t2 WHERE c1 = t1.c1)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1`.`c1`", + "proposal" : "`spark_catalog`.`default`.`t2`.`c1`, `spark_catalog`.`default`.`t2`.`c2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 74, + "stopIndex" : 78, + "fragment" : "t1.c1" + } ] +} + + +-- !query +SELECT * FROM t1 WHERE c1 = (SELECT MIN(a) FROM t2, LATERAL (SELECT c1 AS a)) +-- !query analysis +Project [c1#x, c2#x] ++- Filter (c1#x = scalar-subquery#x []) + : +- Aggregate [min(a#x) AS min(a)#x] + : +- LateralJoin lateral-subquery#x [c1#x], Inner + : : +- SubqueryAlias __auto_generated_subquery_name + : : +- Project [outer(c1#x) AS a#x] + : : +- OneRowRelation + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 WHERE c1 = (SELECT MIN(a) FROM t2, LATERAL (SELECT c1 AS a) WHERE c1 = t1.c1) +-- !query analysis +Project [c1#x, c2#x] ++- Filter (c1#x = scalar-subquery#x [c1#x]) + : +- Aggregate [min(a#x) AS min(a)#x] + : +- Filter (c1#x = outer(c1#x)) + : +- LateralJoin lateral-subquery#x [c1#x], Inner + : : +- SubqueryAlias __auto_generated_subquery_name + : : +- Project [outer(c1#x) AS a#x] + : : +- OneRowRelation + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT COUNT(*) cnt FROM t2 WHERE c1 = t1.c1) +-- !query analysis +Project [c1#x, c2#x, cnt#xL] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [count(1) AS cnt#xL] + : +- Filter (c1#x = outer(c1#x)) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT COUNT(*) cnt, SUM(c2) sum FROM t2 WHERE c1 = t1.c1) +-- !query analysis +Project [c1#x, c2#x, cnt#xL, sum#xL] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [count(1) AS cnt#xL, sum(c2#x) AS sum#xL] + : +- Filter (c1#x = outer(c1#x)) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT SUM(c2) IS NULL FROM t2 WHERE t1.c1 = t2.c1) +-- !query analysis +Project [c1#x, c2#x, (sum(c2) IS NULL)#x] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [isnull(sum(c2#x)) AS (sum(c2) IS NULL)#x] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT COUNT(*) + CASE WHEN sum(c2) IS NULL THEN 0 ELSE sum(c2) END FROM t2 WHERE t1.c1 = t2.c1) +-- !query analysis +Project [c1#x, c2#x, (count(1) + CASE WHEN (sum(c2) IS NULL) THEN 0 ELSE sum(c2) END)#xL] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [(count(1) + CASE WHEN isnull(sum(c2#x)) THEN cast(0 as bigint) ELSE sum(c2#x) END) AS (count(1) + CASE WHEN (sum(c2) IS NULL) THEN 0 ELSE sum(c2) END)#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c1, COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c1) +-- !query analysis +Project [c1#x, c2#x, c1#x, cnt#xL] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [c1#x], [c1#x, count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c2, COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c2) +-- !query analysis +Project [c1#x, c2#x, c2#x, cnt#xL] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [c2#x], [c2#x, count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) +-- !query analysis +Project [c1#x, c2#x, cnt#xL] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 LEFT JOIN LATERAL (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) +-- !query analysis +Project [c1#x, c2#x, cnt#xL] ++- LateralJoin lateral-subquery#x [c1#x], LeftOuter + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 CROSS JOIN LATERAL (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) +-- !query analysis +Project [c1#x, c2#x, cnt#xL] ++- LateralJoin lateral-subquery#x [c1#x], Cross + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 LEFT JOIN LATERAL (SELECT c1, COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c1) +-- !query analysis +Project [c1#x, c2#x, c1#x, cnt#xL] ++- LateralJoin lateral-subquery#x [c1#x], LeftOuter + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [c1#x], [c1#x, count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 CROSS JOIN LATERAL (SELECT c1, COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c1) +-- !query analysis +Project [c1#x, c2#x, c1#x, cnt#xL] ++- LateralJoin lateral-subquery#x [c1#x], Cross + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [c1#x], [c1#x, count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) ON cnt + 1 = c1 +-- !query analysis +Project [c1#x, c2#x, cnt#xL] ++- LateralJoin lateral-subquery#x [c1#x], Inner, ((cnt#xL + cast(1 as bigint)) = cast(c1#x as bigint)) + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT COUNT(*) cnt FROM t1 t2 WHERE t1.c1 = t2.c1) +-- !query analysis +Project [c1#x, c2#x, cnt#xL] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias t2 + : +- SubqueryAlias spark_catalog.default.t1 + : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT COUNT(*) cnt FROM t1 t2 WHERE t1.c1 = t2.c1 HAVING cnt > 0) +-- !query analysis +Project [c1#x, c2#x, cnt#xL] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Filter (cnt#xL > cast(0 as bigint)) + : +- Aggregate [count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias t2 + : +- SubqueryAlias spark_catalog.default.t1 + : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT SUM(cnt) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1)) +-- !query analysis +Project [c1#x, c2#x, sum(cnt)#xL] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [sum(cnt#xL) AS sum(cnt)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT COUNT(cnt) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1)) +-- !query analysis +Project [c1#x, c2#x, count(cnt)#xL] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [count(cnt#xL) AS count(cnt)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT SUM(cnt) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c1)) +-- !query analysis +Project [c1#x, c2#x, sum(cnt)#xL] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [sum(cnt#xL) AS sum(cnt)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [c1#x], [count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL ( + SELECT COUNT(*) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) + JOIN t2 ON cnt = t2.c1 +) +-- !query analysis +Project [c1#x, c2#x, count(1)#xL] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [count(1) AS count(1)#xL] + : +- Join Inner, (cnt#xL = cast(c1#x as bigint)) + : :- SubqueryAlias __auto_generated_subquery_name + : : +- Aggregate [count(1) AS cnt#xL] + : : +- Filter (outer(c1#x) = c1#x) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) WHERE cnt = c1 - 1) +-- !query analysis +Project [c1#x, c2#x, cnt#xL] ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [cnt#xL] + : +- Filter (cnt#xL = cast((outer(c1#x) - 1) as bigint)) + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT COUNT(*) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) WHERE cnt = c1 - 1) +-- !query analysis +Project [c1#x, c2#x, count(1)#xL] ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [count(1) AS count(1)#xL] + : +- Filter (cnt#xL = cast((outer(c1#x) - 1) as bigint)) + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL ( + SELECT COUNT(*) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) + WHERE cnt = c1 - 1 GROUP BY cnt +) +-- !query analysis +Project [c1#x, c2#x, count(1)#xL] ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [cnt#xL], [count(1) AS count(1)#xL] + : +- Filter (cnt#xL = cast((outer(c1#x) - 1) as bigint)) + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL ( + SELECT * FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) + JOIN t2 ON cnt = t2.c1 +) +-- !query analysis +Project [c1#x, c2#x, cnt#xL, c1#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [cnt#xL, c1#x, c2#x] + : +- Join Inner, (cnt#xL = cast(c1#x as bigint)) + : :- SubqueryAlias __auto_generated_subquery_name + : : +- Aggregate [count(1) AS cnt#xL] + : : +- Filter (outer(c1#x) = c1#x) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL ( + SELECT l.cnt + r.cnt + FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) l + JOIN (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) r +) +-- !query analysis +Project [c1#x, c2#x, (cnt + cnt)#xL] ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [(cnt#xL + cnt#xL) AS (cnt + cnt)#xL] + : +- Join Inner + : :- SubqueryAlias l + : : +- Aggregate [count(1) AS cnt#xL] + : : +- Filter (outer(c1#x) = c1#x) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias r + : +- Aggregate [count(1) AS cnt#xL] + : +- Filter (outer(c1#x) = c1#x) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 LEFT JOIN LATERAL (SELECT MIN(c2) FROM t2 WHERE c1 = t1.c1 GROUP BY c1) +-- !query analysis +Project [c1#x, c2#x, min(c2)#x] ++- LateralJoin lateral-subquery#x [c1#x], LeftOuter + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [c1#x], [min(c2#x) AS min(c2)#x] + : +- Filter (c1#x = outer(c1#x)) + : +- SubqueryAlias spark_catalog.default.t2 + : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +WITH cte1 AS ( + SELECT c1 FROM t1 +), cte2 AS ( + SELECT s.* FROM cte1, LATERAL (SELECT * FROM t2 WHERE c1 = cte1.c1) s +) +SELECT * FROM cte2 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte1 +: +- Project [c1#x] +: +- SubqueryAlias spark_catalog.default.t1 +: +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) +: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] +: +- LocalRelation [col1#x, col2#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte2 +: +- Project [c1#x, c2#x] +: +- LateralJoin lateral-subquery#x [c1#x], Inner +: : +- SubqueryAlias s +: : +- Project [c1#x, c2#x] +: : +- Filter (c1#x = outer(c1#x)) +: : +- SubqueryAlias spark_catalog.default.t2 +: : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) +: : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] +: : +- LocalRelation [col1#x, col2#x] +: +- SubqueryAlias cte1 +: +- CTERelationRef xxxx, true, [c1#x] ++- Project [c1#x, c2#x] + +- SubqueryAlias cte2 + +- CTERelationRef xxxx, true, [c1#x, c2#x] + + +-- !query +SELECT * FROM t3 JOIN LATERAL (SELECT EXPLODE(c2)) +-- !query analysis +Project [c1#x, c2#x, col#x] ++- LateralJoin lateral-subquery#x [c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [col#x] + : +- Generate explode(outer(c2#x)), false, [col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t3 + +- View (`spark_catalog`.`default`.`t3`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as array) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t3 JOIN LATERAL (SELECT EXPLODE_OUTER(c2)) +-- !query analysis +Project [c1#x, c2#x, col#x] ++- LateralJoin lateral-subquery#x [c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [col#x] + : +- Generate explode(outer(c2#x)), true, [col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t3 + +- View (`spark_catalog`.`default`.`t3`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as array) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t3 JOIN LATERAL (SELECT EXPLODE(c2)) t(c3) ON c1 = c3 +-- !query analysis +Project [c1#x, c2#x, c3#x] ++- LateralJoin lateral-subquery#x [c2#x], Inner, (c1#x = c3#x) + : +- SubqueryAlias t + : +- Project [col#x AS c3#x] + : +- Project [col#x] + : +- Generate explode(outer(c2#x)), false, [col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t3 + +- View (`spark_catalog`.`default`.`t3`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as array) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t3 LEFT JOIN LATERAL (SELECT EXPLODE(c2)) t(c3) ON c1 = c3 +-- !query analysis +Project [c1#x, c2#x, c3#x] ++- LateralJoin lateral-subquery#x [c2#x], LeftOuter, (c1#x = c3#x) + : +- SubqueryAlias t + : +- Project [col#x AS c3#x] + : +- Project [col#x] + : +- Generate explode(outer(c2#x)), false, [col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t3 + +- View (`spark_catalog`.`default`.`t3`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as array) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT sum(t2.c2) over (order by t2.c1) + FROM t2 + WHERE t2.c1 >= t1.c1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (c1#x >= outer(c1#x))\n+- SubqueryAlias spark_catalog.default.t2\n +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 108, + "fragment" : "SELECT sum(t2.c2) over (order by t2.c1)\n FROM t2\n WHERE t2.c1 >= t1.c1" + } ] +} + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 = t1.c1) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Union false, false + : :- Project [c2#x] + : : +- Filter (c1#x = outer(c1#x)) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- Filter (c1#x = outer(c1#x)) + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION DISTINCT + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Distinct + : +- Union false, false + : :- Project [c2#x] + : : +- Filter (c1#x = outer(c1#x)) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- Filter (c1#x > outer(c2#x)) + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + INTERSECT ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 = t1.c1) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Intersect All true + : :- Project [c2#x] + : : +- Filter (c1#x = outer(c1#x)) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- Filter (c1#x = outer(c1#x)) + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + INTERSECT DISTINCT + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Intersect false + : :- Project [c2#x] + : : +- Filter (c1#x = outer(c1#x)) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- Filter (c1#x > outer(c2#x)) + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + EXCEPT ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 = t1.c1) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Except All true + : :- Project [c2#x] + : : +- Filter (c1#x = outer(c1#x)) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- Filter (c1#x = outer(c1#x)) + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + EXCEPT DISTINCT + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Except false + : :- Project [c2#x] + : : +- Filter (c1#x = outer(c1#x)) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- Filter (c1#x > outer(c2#x)) + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT COUNT(t2.c2) + FROM t2 + WHERE t2.c1 = t1.c1 + UNION DISTINCT + SELECT COUNT(t4.c2) + FROM t4 + WHERE t4.c1 > t1.c2) +-- !query analysis +Project [c1#x, c2#x, count(c2)#xL] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Distinct + : +- Union false, false + : :- Aggregate [count(c2#x) AS count(c2)#xL] + : : +- Filter (c1#x = outer(c1#x)) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Aggregate [count(c2#x) AS count(c2)#xL] + : +- Filter (c1#x > outer(c2#x)) + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c1, t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION ALL + SELECT t4.c2, t4.c1 + FROM t4 + WHERE t4.c1 = t1.c1) +-- !query analysis +Project [c1#x, c2#x, c1#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Union false, false + : :- Project [c1#x, c2#x] + : : +- Filter (c1#x = outer(c1#x)) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x, c1#x] + : +- Filter (c1#x = outer(c1#x)) + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + UNION ALL + SELECT t4.c2 + FROM t4) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Union false, false + : :- Project [c2#x] + : : +- Filter ((c1#x = outer(c1#x)) AND (c2#x >= outer(c2#x))) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Union false, false + : :- Project [c2#x] + : : +- Filter (c1#x = outer(c1#x)) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + UNION DISTINCT + SELECT t4.c2 + FROM t4) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Distinct + : +- Union false, false + : :- Project [c2#x] + : : +- Filter ((c1#x = outer(c1#x)) AND (c2#x >= outer(c2#x))) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + INTERSECT ALL + SELECT t4.c2 + FROM t4) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Intersect All true + : :- Project [c2#x] + : : +- Filter ((c1#x = outer(c1#x)) AND (c2#x >= outer(c2#x))) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + INTERSECT DISTINCT + SELECT t4.c2 + FROM t4) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Intersect false + : :- Project [c2#x] + : : +- Filter ((c1#x = outer(c1#x)) AND (c2#x >= outer(c2#x))) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + EXCEPT ALL + SELECT t4.c2 + FROM t4) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Except All true + : :- Project [c2#x] + : : +- Filter ((c1#x = outer(c1#x)) AND (c2#x >= outer(c2#x))) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + EXCEPT DISTINCT + SELECT t4.c2 + FROM t4) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Except false + : :- Project [c2#x] + : : +- Filter ((c1#x = outer(c1#x)) AND (c2#x >= outer(c2#x))) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + GROUP BY t2.c2 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2 + GROUP BY t4.c2) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Union false, false + : :- Aggregate [c2#x], [c2#x] + : : +- Filter (c1#x = outer(c1#x)) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Aggregate [c2#x], [c2#x] + : +- Filter (c1#x > outer(c2#x)) + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c1 - t1.c1 + FROM t2 + GROUP BY t2.c1 - t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2 + GROUP BY t4.c2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"(c1 - c1)\",\"(c1 - c1) AS `(c1 - outer(spark_catalog.default.t1.c1))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 69, + "stopIndex" : 90, + "fragment" : "GROUP BY t2.c1 - t1.c1" + } ] +} + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT sum(t2.c2) over (order by t2.c1) + FROM t2 + WHERE t2.c1 >= t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (c1#x >= outer(c1#x))\n+- SubqueryAlias spark_catalog.default.t2\n +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 108, + "fragment" : "SELECT sum(t2.c2) over (order by t2.c1)\n FROM t2\n WHERE t2.c1 >= t1.c1" + } ] +} + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT * FROM t2 WHERE t2.c1 = t1.c1) +UNION ALL +SELECT * FROM t1 JOIN t4 +-- !query analysis +Union false, false +:- Project [c1#x, c2#x, c1#x, c2#x] +: +- LateralJoin lateral-subquery#x [c1#x], Inner +: : +- SubqueryAlias __auto_generated_subquery_name +: : +- Project [c1#x, c2#x] +: : +- Filter (c1#x = outer(c1#x)) +: : +- SubqueryAlias spark_catalog.default.t2 +: : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) +: : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] +: : +- LocalRelation [col1#x, col2#x] +: +- SubqueryAlias spark_catalog.default.t1 +: +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) +: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] +: +- LocalRelation [col1#x, col2#x] ++- Project [c1#x AS c1#x, c2#x AS c2#x, c1#x AS c1#x, c2#x AS c2#x] + +- Project [c1#x, c2#x, c1#x, c2#x] + +- Join Inner + :- SubqueryAlias spark_catalog.default.t1 + : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t4 + +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 = t1.c1) +UNION ALL +SELECT * FROM t2 JOIN LATERAL + (SELECT t1.c2 + FROM t1 + WHERE t2.c1 <= t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 < t2.c1) +-- !query analysis +Union false, false +:- Project [c1#x, c2#x, c2#x] +: +- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner +: : +- SubqueryAlias __auto_generated_subquery_name +: : +- Union false, false +: : :- Project [c2#x] +: : : +- Filter (c1#x = outer(c1#x)) +: : : +- SubqueryAlias spark_catalog.default.t2 +: : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) +: : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] +: : : +- LocalRelation [col1#x, col2#x] +: : +- Project [c2#x] +: : +- Filter (c1#x = outer(c1#x)) +: : +- SubqueryAlias spark_catalog.default.t4 +: : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) +: : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] +: : +- LocalRelation [col1#x, col2#x] +: +- SubqueryAlias spark_catalog.default.t1 +: +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) +: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] +: +- LocalRelation [col1#x, col2#x] ++- Project [c1#x AS c1#x, c2#x AS c2#x, c2#x AS c2#x] + +- Project [c1#x, c2#x, c2#x] + +- LateralJoin lateral-subquery#x [c1#x && c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Union false, false + : :- Project [c2#x] + : : +- Filter (outer(c1#x) <= c1#x) + : : +- SubqueryAlias spark_catalog.default.t1 + : : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- Filter (c1#x < outer(c1#x)) + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t2 + +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + ((SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + EXCEPT DISTINCT + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2) + UNION DISTINCT + (SELECT t4.c1 + FROM t4 + WHERE t4.c1 <= t1.c2 + INTERSECT ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 <> t1.c1) +) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x && c2#x && c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Distinct + : +- Union false, false + : :- Except false + : : :- Project [c2#x] + : : : +- Filter (c1#x = outer(c1#x)) + : : : +- SubqueryAlias spark_catalog.default.t2 + : : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : : +- LocalRelation [col1#x, col2#x] + : : +- Project [c2#x] + : : +- Filter (c1#x > outer(c2#x)) + : : +- SubqueryAlias spark_catalog.default.t4 + : : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Intersect All true + : :- Project [c1#x] + : : +- Filter (c1#x <= outer(c2#x)) + : : +- SubqueryAlias spark_catalog.default.t4 + : : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- Filter NOT (c1#x = outer(c1#x)) + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL + ((SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2) + INTERSECT DISTINCT + (SELECT t4.c1 + FROM t4 + WHERE t4.c1 <= t1.c2 + EXCEPT ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 <> t1.c1) +) +-- !query analysis +Project [c1#x, c2#x, c2#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x && c2#x && c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Intersect false + : :- Union false, false + : : :- Project [c2#x] + : : : +- Filter (c1#x = outer(c1#x)) + : : : +- SubqueryAlias spark_catalog.default.t2 + : : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : : +- LocalRelation [col1#x, col2#x] + : : +- Project [c2#x] + : : +- Filter (c1#x > outer(c2#x)) + : : +- SubqueryAlias spark_catalog.default.t4 + : : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Except All true + : :- Project [c1#x] + : : +- Filter (c1#x <= outer(c2#x)) + : : +- SubqueryAlias spark_catalog.default.t4 + : : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- Project [c2#x] + : +- Filter NOT (c1#x = outer(c1#x)) + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT sum(c1) FROM + (SELECT * + FROM t2 + WHERE t2.c1 <= t1.c1) lhs + LEFT SEMI JOIN + (SELECT * + FROM t4) rhs + ON lhs.c1 <=> rhs.c1 and lhs.c2 <=> rhs.c2 +) +-- !query analysis +Project [c1#x, c2#x, sum(c1)#xL] ++- LateralJoin lateral-subquery#x [c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [sum(c1#x) AS sum(c1)#xL] + : +- Join LeftSemi, ((c1#x <=> c1#x) AND (c2#x <=> c2#x)) + : :- SubqueryAlias lhs + : : +- Project [c1#x, c2#x] + : : +- Filter (c1#x <= outer(c1#x)) + : : +- SubqueryAlias spark_catalog.default.t2 + : : +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x]) + : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias rhs + : +- Project [c1#x, c2#x] + : +- SubqueryAlias spark_catalog.default.t4 + : +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT sum(c1) FROM + (SELECT * + FROM t2 + WHERE t2.c1 <= t1.c1) lhs + LEFT SEMI JOIN + (SELECT * + FROM t4 + WHERE t4.c1 > t1.c2) rhs + ON lhs.c1 <=> rhs.c1 and lhs.c2 <=> rhs.c2 +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (c1#x > outer(c2#x))\n+- SubqueryAlias spark_catalog.default.t4\n +- View (`spark_catalog`.`default`.`t4`, [c1#x,c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 125, + "stopIndex" : 166, + "fragment" : "SELECT *\n FROM t4\n WHERE t4.c1 > t1.c2" + } ] +} + + +-- !query +SELECT * FROM LATERAL EXPLODE(ARRAY(1, 2)) +-- !query analysis +Project [col#x] ++- Generate explode(array(1, 2)), false, [col#x] + +- OneRowRelation + + +-- !query +SELECT * FROM t1, LATERAL RANGE(3) +-- !query analysis +Project [c1#x, c2#x, id#xL] ++- LateralJoin lateral-subquery#x [], Inner + : +- Range (0, 3, step=1, splits=None) + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3) +-- !query analysis +Project [c1#x, c2#x, c3#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias t2 + : +- Project [col#x AS c3#x] + : +- Generate explode(array(outer(c1#x), outer(c2#x))), false, [col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t3, LATERAL EXPLODE(c2) t2(v) +-- !query analysis +Project [c1#x, c2#x, v#x] ++- LateralJoin lateral-subquery#x [c2#x], Inner + : +- SubqueryAlias t2 + : +- Project [col#x AS v#x] + : +- Generate explode(outer(c2#x)), false, [col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t3 + +- View (`spark_catalog`.`default`.`t3`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as array) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v) +-- !query analysis +Project [c1#x, c2#x, v#x] ++- LateralJoin lateral-subquery#x [c2#x], Inner + : +- SubqueryAlias t2 + : +- Project [col#x AS v#x] + : +- Generate explode(outer(c2#x)), true, [col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t3 + +- View (`spark_catalog`.`default`.`t3`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as array) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM EXPLODE(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1) +-- !query analysis +Project [v#x, (outer(t.v) + 1)#x] ++- LateralJoin lateral-subquery#x [v#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [(outer(v#x) + 1) AS (outer(t.v) + 1)#x] + : +- OneRowRelation + +- SubqueryAlias t + +- Project [col#x AS v#x] + +- Generate explode(array(1, 2)), false, [col#x] + +- OneRowRelation + + +-- !query +SELECT * FROM t1 JOIN LATERAL EXPLODE(ARRAY(c1, c2)) t(c3) ON t1.c1 = c3 +-- !query analysis +Project [c1#x, c2#x, c3#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner, (c1#x = c3#x) + : +- SubqueryAlias t + : +- Project [col#x AS c3#x] + : +- Generate explode(array(outer(c1#x), outer(c2#x))), false, [col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t3 JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3 +-- !query analysis +Project [c1#x, c2#x, c3#x] ++- LateralJoin lateral-subquery#x [c2#x], Inner, (c1#x = c3#x) + : +- SubqueryAlias t + : +- Project [col#x AS c3#x] + : +- Generate explode(outer(c2#x)), false, [col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t3 + +- View (`spark_catalog`.`default`.`t3`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as array) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t3 LEFT JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3 +-- !query analysis +Project [c1#x, c2#x, c3#x] ++- LateralJoin lateral-subquery#x [c2#x], LeftOuter, (c1#x = c3#x) + : +- SubqueryAlias t + : +- Project [col#x AS c3#x] + : +- Generate explode(outer(c2#x)), false, [col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t3 + +- View (`spark_catalog`.`default`.`t3`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as array) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM EXPLODE(ARRAY(c1, c2))) +-- !query analysis +Project [c1#x, c2#x, col#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [col#x] + : +- Generate explode(array(outer(c1#x), outer(c2#x))), false, [col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.c1 + c3 FROM EXPLODE(ARRAY(c1, c2)) t(c3)) +-- !query analysis +Project [c1#x, c2#x, (outer(spark_catalog.default.t1.c1) + c3)#x] ++- LateralJoin lateral-subquery#x [c1#x && c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [(outer(c1#x) + c3#x) AS (outer(spark_catalog.default.t1.c1) + c3)#x] + : +- SubqueryAlias t + : +- Project [col#x AS c3#x] + : +- Generate explode(array(outer(c1#x), outer(c2#x))), false, [col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.c1 + c3 FROM EXPLODE(ARRAY(c1, c2)) t(c3) WHERE t1.c2 > 1) +-- !query analysis +Project [c1#x, c2#x, (outer(spark_catalog.default.t1.c1) + c3)#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x && c1#x && c2#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [(outer(c1#x) + c3#x) AS (outer(spark_catalog.default.t1.c1) + c3)#x] + : +- Filter (outer(c2#x) > 1) + : +- SubqueryAlias t + : +- Project [col#x AS c3#x] + : +- Generate explode(array(outer(c1#x), outer(c2#x))), false, [col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM EXPLODE(ARRAY(c1, c2)) l(x) JOIN EXPLODE(ARRAY(c2, c1)) r(y) ON x = y) +-- !query analysis +Project [c1#x, c2#x, x#x, y#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x && c2#x && c1#x], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [x#x, y#x] + : +- Join Inner, (x#x = y#x) + : :- SubqueryAlias l + : : +- Project [col#x AS x#x] + : : +- Generate explode(array(outer(c1#x), outer(c2#x))), false, [col#x] + : : +- OneRowRelation + : +- SubqueryAlias r + : +- Project [col#x AS y#x] + : +- Generate explode(array(outer(c2#x), outer(c1#x))), false, [col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW array_struct(id, arr) AS VALUES + (1, ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))), + (2, ARRAY()), + (3, ARRAY(STRUCT(3, 'c'))) +-- !query analysis +CreateViewCommand `array_struct`, [(id,None), (arr,None)], VALUES + (1, ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))), + (2, ARRAY()), + (3, ARRAY(STRUCT(3, 'c'))), false, true, LocalTempView, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1, LATERAL INLINE(ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))) +-- !query analysis +Project [c1#x, c2#x, col1#x, col2#x] ++- LateralJoin lateral-subquery#x [], Inner + : +- Generate inline(array(struct(col1, 1, col2, a), struct(col1, 2, col2, b))), false, [col1#x, col2#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT c1, t.* FROM t1, LATERAL INLINE(ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))) t(x, y) +-- !query analysis +Project [c1#x, x#x, y#x] ++- LateralJoin lateral-subquery#x [], Inner + : +- SubqueryAlias t + : +- Project [col1#x AS x#x, col2#x AS y#x] + : +- Generate inline(array(struct(col1, 1, col2, a), struct(col1, 2, col2, b))), false, [col1#x, col2#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM array_struct JOIN LATERAL INLINE(arr) +-- !query analysis +Project [id#x, arr#x, col1#x, col2#x] ++- LateralJoin lateral-subquery#x [arr#x], Inner + : +- Generate inline(outer(arr#x)), false, [col1#x, col2#x] + : +- OneRowRelation + +- SubqueryAlias array_struct + +- View (`array_struct`, [id#x,arr#x]) + +- Project [cast(col1#x as int) AS id#x, cast(col2#x as array>) AS arr#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM array_struct LEFT JOIN LATERAL INLINE(arr) t(k, v) ON id = k +-- !query analysis +Project [id#x, arr#x, k#x, v#x] ++- LateralJoin lateral-subquery#x [arr#x], LeftOuter, (id#x = k#x) + : +- SubqueryAlias t + : +- Project [col1#x AS k#x, col2#x AS v#x] + : +- Generate inline(outer(arr#x)), false, [col1#x, col2#x] + : +- OneRowRelation + +- SubqueryAlias array_struct + +- View (`array_struct`, [id#x,arr#x]) + +- Project [cast(col1#x as int) AS id#x, cast(col2#x as array>) AS arr#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM array_struct JOIN LATERAL INLINE_OUTER(arr) +-- !query analysis +Project [id#x, arr#x, col1#x, col2#x] ++- LateralJoin lateral-subquery#x [arr#x], Inner + : +- Generate inline(outer(arr#x)), true, [col1#x, col2#x] + : +- OneRowRelation + +- SubqueryAlias array_struct + +- View (`array_struct`, [id#x,arr#x]) + +- Project [cast(col1#x as int) AS id#x, cast(col2#x as array>) AS arr#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +DROP VIEW array_struct +-- !query analysis +DropTempViewCommand array_struct + + +-- !query +SELECT * FROM LATERAL posexplode(ARRAY(1, 2)) +-- !query analysis +Project [pos#x, col#x] ++- Generate posexplode(array(1, 2)), false, [pos#x, col#x] + +- OneRowRelation + + +-- !query +SELECT * FROM t1, LATERAL posexplode(ARRAY(c1, c2)) t2(pos, c3) +-- !query analysis +Project [c1#x, c2#x, pos#x, c3#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias t2 + : +- Project [pos#x AS pos#x, col#x AS c3#x] + : +- Generate posexplode(array(outer(c1#x), outer(c2#x))), false, [pos#x, col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 JOIN LATERAL posexplode(ARRAY(c1, c2)) t(pos, c3) ON t1.c1 = c3 +-- !query analysis +Project [c1#x, c2#x, pos#x, c3#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner, (c1#x = c3#x) + : +- SubqueryAlias t + : +- Project [pos#x AS pos#x, col#x AS c3#x] + : +- Generate posexplode(array(outer(c1#x), outer(c2#x))), false, [pos#x, col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t3, LATERAL posexplode(c2) t2(pos, v) +-- !query analysis +Project [c1#x, c2#x, pos#x, v#x] ++- LateralJoin lateral-subquery#x [c2#x], Inner + : +- SubqueryAlias t2 + : +- Project [pos#x AS pos#x, col#x AS v#x] + : +- Generate posexplode(outer(c2#x)), false, [pos#x, col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t3 + +- View (`spark_catalog`.`default`.`t3`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as array) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t3 JOIN LATERAL posexplode(c2) t(pos, c3) ON t3.c1 = c3 +-- !query analysis +Project [c1#x, c2#x, pos#x, c3#x] ++- LateralJoin lateral-subquery#x [c2#x], Inner, (c1#x = c3#x) + : +- SubqueryAlias t + : +- Project [pos#x AS pos#x, col#x AS c3#x] + : +- Generate posexplode(outer(c2#x)), false, [pos#x, col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t3 + +- View (`spark_catalog`.`default`.`t3`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as array) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t3, LATERAL posexplode_outer(c2) t2(pos, v) +-- !query analysis +Project [c1#x, c2#x, pos#x, v#x] ++- LateralJoin lateral-subquery#x [c2#x], Inner + : +- SubqueryAlias t2 + : +- Project [pos#x AS pos#x, col#x AS v#x] + : +- Generate posexplode(outer(c2#x)), true, [pos#x, col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t3 + +- View (`spark_catalog`.`default`.`t3`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as array) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t3 LEFT JOIN LATERAL posexplode(c2) t(pos, c3) ON t3.c1 = c3 +-- !query analysis +Project [c1#x, c2#x, pos#x, c3#x] ++- LateralJoin lateral-subquery#x [c2#x], LeftOuter, (c1#x = c3#x) + : +- SubqueryAlias t + : +- Project [pos#x AS pos#x, col#x AS c3#x] + : +- Generate posexplode(outer(c2#x)), false, [pos#x, col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t3 + +- View (`spark_catalog`.`default`.`t3`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as array) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t3 LEFT JOIN LATERAL posexplode_outer(c2) t(pos, c3) ON t3.c1 = c3 +-- !query analysis +Project [c1#x, c2#x, pos#x, c3#x] ++- LateralJoin lateral-subquery#x [c2#x], LeftOuter, (c1#x = c3#x) + : +- SubqueryAlias t + : +- Project [pos#x AS pos#x, col#x AS c3#x] + : +- Generate posexplode(outer(c2#x)), true, [pos#x, col#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t3 + +- View (`spark_catalog`.`default`.`t3`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as array) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE OR REPLACE TEMP VIEW json_table(key, jstring) AS VALUES + ('1', '{"f1": "1", "f2": "2", "f3": 3, "f5": 5.23}'), + ('2', '{"f1": "1", "f3": "3", "f2": 2, "f4": 4.01}'), + ('3', '{"f1": 3, "f4": "4", "f3": "3", "f2": 2, "f5": 5.01}'), + ('4', cast(null as string)), + ('5', '{"f1": null, "f5": ""}'), + ('6', '[invalid JSON string]') +-- !query analysis +CreateViewCommand `json_table`, [(key,None), (jstring,None)], VALUES + ('1', '{"f1": "1", "f2": "2", "f3": 3, "f5": 5.23}'), + ('2', '{"f1": "1", "f3": "3", "f2": 2, "f4": 4.01}'), + ('3', '{"f1": 3, "f4": "4", "f3": "3", "f2": 2, "f5": 5.01}'), + ('4', cast(null as string)), + ('5', '{"f1": null, "f5": ""}'), + ('6', '[invalid JSON string]'), false, true, LocalTempView, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t1.key, t2.* FROM json_table t1, LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2 +-- !query analysis +Project [key#x, c0#x, c1#x, c2#x, c3#x, c4#x] ++- LateralJoin lateral-subquery#x [jstring#x], Inner + : +- SubqueryAlias t2 + : +- Generate json_tuple(outer(jstring#x), f1, f2, f3, f4, f5), false, [c0#x, c1#x, c2#x, c3#x, c4#x] + : +- OneRowRelation + +- SubqueryAlias t1 + +- SubqueryAlias json_table + +- View (`json_table`, [key#x,jstring#x]) + +- Project [cast(col1#x as string) AS key#x, cast(col2#x as string) AS jstring#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t1.key, t2.* FROM json_table t1, LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2 WHERE t2.c0 IS NOT NULL +-- !query analysis +Project [key#x, c0#x, c1#x, c2#x, c3#x, c4#x] ++- Filter isnotnull(c0#x) + +- LateralJoin lateral-subquery#x [jstring#x], Inner + : +- SubqueryAlias t2 + : +- Generate json_tuple(outer(jstring#x), f1, f2, f3, f4, f5), false, [c0#x, c1#x, c2#x, c3#x, c4#x] + : +- OneRowRelation + +- SubqueryAlias t1 + +- SubqueryAlias json_table + +- View (`json_table`, [key#x,jstring#x]) + +- Project [cast(col1#x as string) AS key#x, cast(col2#x as string) AS jstring#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t1.key, t2.* FROM json_table t1 + JOIN LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2(f1, f2, f3, f4, f5) + ON t1.key = t2.f1 +-- !query analysis +Project [key#x, f1#x, f2#x, f3#x, f4#x, f5#x] ++- LateralJoin lateral-subquery#x [jstring#x], Inner, (key#x = f1#x) + : +- SubqueryAlias t2 + : +- Project [c0#x AS f1#x, c1#x AS f2#x, c2#x AS f3#x, c3#x AS f4#x, c4#x AS f5#x] + : +- Generate json_tuple(outer(jstring#x), f1, f2, f3, f4, f5), false, [c0#x, c1#x, c2#x, c3#x, c4#x] + : +- OneRowRelation + +- SubqueryAlias t1 + +- SubqueryAlias json_table + +- View (`json_table`, [key#x,jstring#x]) + +- Project [cast(col1#x as string) AS key#x, cast(col2#x as string) AS jstring#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t1.key, t2.* FROM json_table t1 + LEFT JOIN LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2(f1, f2, f3, f4, f5) + ON t1.key = t2.f1 +-- !query analysis +Project [key#x, f1#x, f2#x, f3#x, f4#x, f5#x] ++- LateralJoin lateral-subquery#x [jstring#x], LeftOuter, (key#x = f1#x) + : +- SubqueryAlias t2 + : +- Project [c0#x AS f1#x, c1#x AS f2#x, c2#x AS f3#x, c3#x AS f4#x, c4#x AS f5#x] + : +- Generate json_tuple(outer(jstring#x), f1, f2, f3, f4, f5), false, [c0#x, c1#x, c2#x, c3#x, c4#x] + : +- OneRowRelation + +- SubqueryAlias t1 + +- SubqueryAlias json_table + +- View (`json_table`, [key#x,jstring#x]) + +- Project [cast(col1#x as string) AS key#x, cast(col2#x as string) AS jstring#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +DROP VIEW json_table +-- !query analysis +DropTempViewCommand json_table + + +-- !query +SELECT t.* FROM t1, LATERAL stack(2, 'Key', c1, 'Value', c2) t +-- !query analysis +Project [col0#x, col1#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias t + : +- Generate stack(2, Key, outer(c1#x), Value, outer(c2#x)), false, [col0#x, col1#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t(x, y) +-- !query analysis +Project [x#x, y#x] ++- LateralJoin lateral-subquery#x [c1#x && c2#x], Inner + : +- SubqueryAlias t + : +- Project [col0#x AS x#x, col1#x AS y#x] + : +- Generate stack(1, outer(c1#x), outer(c2#x)), false, [col0#x, col1#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.default.t1 + +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t.* FROM t1 JOIN t3 ON t1.c1 = t3.c1 JOIN LATERAL stack(1, t1.c2, t3.c2) t +-- !query analysis +Project [col0#x, col1#x] ++- LateralJoin lateral-subquery#x [c2#x && c2#x], Inner + : +- SubqueryAlias t + : +- Generate stack(1, outer(c2#x), outer(c2#x)), false, [col0#x, col1#x] + : +- OneRowRelation + +- Join Inner, (c1#x = c1#x) + :- SubqueryAlias spark_catalog.default.t1 + : +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x]) + : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias spark_catalog.default.t3 + +- View (`spark_catalog`.`default`.`t3`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as array) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t.* FROM t1, LATERAL stack(c1, c2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"outer(spark_catalog.default.t1.c1)\"", + "inputName" : "n", + "inputType" : "\"INT\"", + "sqlExpr" : "\"stack(outer(spark_catalog.default.t1.c1), outer(spark_catalog.default.t1.c2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 41, + "fragment" : "stack(c1, c2)" + } ] +} + + +-- !query +DROP VIEW t1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t1`, false, true, false + + +-- !query +DROP VIEW t2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t2`, false, true, false + + +-- !query +DROP VIEW t3 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t3`, false, true, false + + +-- !query +DROP VIEW t4 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t4`, false, true, false 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 new file mode 100644 index 0000000000000..a3872b4ebc534 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/json-functions.sql.out @@ -0,0 +1,724 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select to_json(named_struct('a', 1, 'b', 2)) +-- !query analysis +Project [to_json(named_struct(a, 1, b, 2), Some(America/Los_Angeles)) AS to_json(named_struct(a, 1, b, 2))#x] ++- OneRowRelation + + +-- !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] ++- OneRowRelation + + +-- !query +select to_json(array(named_struct('a', 1, 'b', 2))) +-- !query analysis +Project [to_json(array(named_struct(a, 1, b, 2)), Some(America/Los_Angeles)) AS to_json(array(named_struct(a, 1, b, 2)))#x] ++- OneRowRelation + + +-- !query +select to_json(map(named_struct('a', 1, 'b', 2), named_struct('a', 1, 'b', 2))) +-- !query analysis +Project [to_json(map(named_struct(a, 1, b, 2), named_struct(a, 1, b, 2)), Some(America/Los_Angeles)) AS to_json(map(named_struct(a, 1, b, 2), named_struct(a, 1, b, 2)))#x] ++- OneRowRelation + + +-- !query +select to_json(map('a', named_struct('a', 1, 'b', 2))) +-- !query analysis +Project [to_json(map(a, named_struct(a, 1, b, 2)), Some(America/Los_Angeles)) AS to_json(map(a, named_struct(a, 1, b, 2)))#x] ++- OneRowRelation + + +-- !query +select to_json(map('a', 1)) +-- !query analysis +Project [to_json(map(a, 1), Some(America/Los_Angeles)) AS to_json(map(a, 1))#x] ++- OneRowRelation + + +-- !query +select to_json(array(map('a',1))) +-- !query analysis +Project [to_json(array(map(a, 1)), Some(America/Los_Angeles)) AS to_json(array(map(a, 1)))#x] ++- OneRowRelation + + +-- !query +select to_json(array(map('a',1), map('b',2))) +-- !query analysis +Project [to_json(array(map(a, 1), map(b, 2)), Some(America/Los_Angeles)) AS to_json(array(map(a, 1), map(b, 2)))#x] ++- OneRowRelation + + +-- !query +select to_json(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_MAP_FUNCTION", + "sqlState" : "42K06", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "to_json(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE'))" + } ] +} + + +-- !query +select to_json(named_struct('a', 1, 'b', 2), map('mode', 1)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_STRING_TYPE", + "sqlState" : "42K06", + "messageParameters" : { + "mapType" : "\"MAP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "to_json(named_struct('a', 1, 'b', 2), map('mode', 1))" + } ] +} + + +-- !query +select to_json() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[1, 2]", + "functionName" : "`to_json`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "to_json()" + } ] +} + + +-- !query +select from_json('{"a":1}', 'a INT') +-- !query analysis +Project [from_json(StructField(a,IntegerType,true), {"a":1}, Some(America/Los_Angeles)) AS from_json({"a":1})#x] ++- OneRowRelation + + +-- !query +select from_json('{"time":"26/08/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')) +-- !query analysis +Project [from_json(StructField(time,TimestampType,true), (timestampFormat,dd/MM/yyyy), {"time":"26/08/2015"}, Some(America/Los_Angeles)) AS from_json({"time":"26/08/2015"})#x] ++- OneRowRelation + + +-- !query +select from_json('{"a":1}', 1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_SCHEMA.NON_STRING_LITERAL", + "sqlState" : "42K07", + "messageParameters" : { + "inputSchema" : "\"1\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "from_json('{\"a\":1}', 1)" + } ] +} + + +-- !query +select from_json('{"a":1}', 'a InvalidType') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'InvalidType'", + "hint" : ": extra input 'InvalidType'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "from_json('{\"a\":1}', 'a InvalidType')" + } ] +} + + +-- !query +select from_json('{"a":1}', 'a INT', named_struct('mode', 'PERMISSIVE')) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_MAP_FUNCTION", + "sqlState" : "42K06", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "from_json('{\"a\":1}', 'a INT', named_struct('mode', 'PERMISSIVE'))" + } ] +} + + +-- !query +select from_json('{"a":1}', 'a INT', map('mode', 1)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_STRING_TYPE", + "sqlState" : "42K06", + "messageParameters" : { + "mapType" : "\"MAP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "from_json('{\"a\":1}', 'a INT', map('mode', 1))" + } ] +} + + +-- !query +select from_json() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3]", + "functionName" : "`from_json`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "from_json()" + } ] +} + + +-- !query +SELECT json_tuple('{"a" : 1, "b" : 2}', CAST(NULL AS STRING), 'b', CAST(NULL AS STRING), 'a') +-- !query analysis +Project [c0#x, c1#x, c2#x, c3#x] ++- Generate json_tuple({"a" : 1, "b" : 2}, cast(null as string), b, cast(null as string), a), false, [c0#x, c1#x, c2#x, c3#x] + +- OneRowRelation + + +-- !query +CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a') +-- !query analysis +CreateViewCommand `jsonTable`, [(jsonField,None), (a,None)], SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a'), false, false, LocalTempView, true + +- Project [col1#x, col2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT json_tuple(jsonField, 'b', CAST(NULL AS STRING), a) FROM jsonTable +-- !query analysis +Project [c0#x, c1#x, c2#x] ++- Generate json_tuple(jsonField#x, b, cast(null as string), a#x), false, [c0#x, c1#x, c2#x] + +- SubqueryAlias jsontable + +- View (`jsonTable`, [jsonField#x,a#x]) + +- Project [cast(col1#x as string) AS jsonField#x, cast(col2#x as string) AS a#x] + +- Project [col1#x, col2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT json_tuple('{"a":"1"}', if(c1 < 1, null, 'a')) FROM ( SELECT rand() AS c1 ) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT json_tuple('{"a":"1"}', if(c1 < 1, null, 'a'), if(c2 < 1, null, 'a')) FROM ( SELECT 0 AS c1, rand() AS c2 ) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +DROP VIEW IF EXISTS jsonTable +-- !query analysis +DropTempViewCommand jsonTable + + +-- !query +select from_json('{"a":1, "b":2}', 'map') +-- !query analysis +Project [from_json(MapType(StringType,IntegerType,true), {"a":1, "b":2}, Some(America/Los_Angeles)) AS entries#x] ++- OneRowRelation + + +-- !query +select from_json('{"a":1, "b":"2"}', 'struct') +-- !query analysis +Project [from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), {"a":1, "b":"2"}, Some(America/Los_Angeles)) AS from_json({"a":1, "b":"2"})#x] ++- OneRowRelation + + +-- !query +select schema_of_json('{"c1":0, "c2":[1]}') +-- !query analysis +Project [schema_of_json({"c1":0, "c2":[1]}) AS schema_of_json({"c1":0, "c2":[1]})#x] ++- OneRowRelation + + +-- !query +select from_json('{"c1":[1, 2, 3]}', schema_of_json('{"c1":[0]}')) +-- !query analysis +Project [from_json(StructField(c1,ArrayType(LongType,true),true), {"c1":[1, 2, 3]}, Some(America/Los_Angeles)) AS from_json({"c1":[1, 2, 3]})#x] ++- OneRowRelation + + +-- !query +select from_json('[1, 2, 3]', 'array') +-- !query analysis +Project [from_json(ArrayType(IntegerType,true), [1, 2, 3], Some(America/Los_Angeles)) AS from_json([1, 2, 3])#x] ++- OneRowRelation + + +-- !query +select from_json('[1, "2", 3]', 'array') +-- !query analysis +Project [from_json(ArrayType(IntegerType,true), [1, "2", 3], Some(America/Los_Angeles)) AS from_json([1, "2", 3])#x] ++- OneRowRelation + + +-- !query +select from_json('[1, 2, null]', 'array') +-- !query analysis +Project [from_json(ArrayType(IntegerType,true), [1, 2, null], Some(America/Los_Angeles)) AS from_json([1, 2, null])#x] ++- OneRowRelation + + +-- !query +select from_json('[{"a": 1}, {"a":2}]', 'array>') +-- !query analysis +Project [from_json(ArrayType(StructType(StructField(a,IntegerType,true)),true), [{"a": 1}, {"a":2}], Some(America/Los_Angeles)) AS from_json([{"a": 1}, {"a":2}])#x] ++- OneRowRelation + + +-- !query +select from_json('{"a": 1}', 'array>') +-- !query analysis +Project [from_json(ArrayType(StructType(StructField(a,IntegerType,true)),true), {"a": 1}, Some(America/Los_Angeles)) AS from_json({"a": 1})#x] ++- OneRowRelation + + +-- !query +select from_json('[null, {"a":2}]', 'array>') +-- !query analysis +Project [from_json(ArrayType(StructType(StructField(a,IntegerType,true)),true), [null, {"a":2}], Some(America/Los_Angeles)) AS from_json([null, {"a":2}])#x] ++- OneRowRelation + + +-- !query +select from_json('[{"a": 1}, {"b":2}]', 'array>') +-- !query analysis +Project [from_json(ArrayType(MapType(StringType,IntegerType,true),true), [{"a": 1}, {"b":2}], Some(America/Los_Angeles)) AS from_json([{"a": 1}, {"b":2}])#x] ++- OneRowRelation + + +-- !query +select from_json('[{"a": 1}, 2]', 'array>') +-- !query analysis +Project [from_json(ArrayType(MapType(StringType,IntegerType,true),true), [{"a": 1}, 2], Some(America/Los_Angeles)) AS from_json([{"a": 1}, 2])#x] ++- OneRowRelation + + +-- !query +select from_json('{"d": "2012-12-15", "t": "2012-12-15 15:15:15"}', 'd date, t timestamp') +-- !query analysis +Project [from_json(StructField(d,DateType,true), StructField(t,TimestampType,true), {"d": "2012-12-15", "t": "2012-12-15 15:15:15"}, Some(America/Los_Angeles)) AS from_json({"d": "2012-12-15", "t": "2012-12-15 15:15:15"})#x] ++- OneRowRelation + + +-- !query +select from_json( + '{"d": "12/15 2012", "t": "12/15 2012 15:15:15"}', + 'd date, t timestamp', + map('dateFormat', 'MM/dd yyyy', 'timestampFormat', 'MM/dd yyyy HH:mm:ss')) +-- !query analysis +Project [from_json(StructField(d,DateType,true), StructField(t,TimestampType,true), (dateFormat,MM/dd yyyy), (timestampFormat,MM/dd yyyy HH:mm:ss), {"d": "12/15 2012", "t": "12/15 2012 15:15:15"}, Some(America/Los_Angeles)) AS from_json({"d": "12/15 2012", "t": "12/15 2012 15:15:15"})#x] ++- OneRowRelation + + +-- !query +select from_json( + '{"d": "02-29"}', + 'd date', + map('dateFormat', 'MM-dd')) +-- !query analysis +Project [from_json(StructField(d,DateType,true), (dateFormat,MM-dd), {"d": "02-29"}, Some(America/Los_Angeles)) AS from_json({"d": "02-29"})#x] ++- OneRowRelation + + +-- !query +select from_json( + '{"t": "02-29"}', + 't timestamp', + map('timestampFormat', 'MM-dd')) +-- !query analysis +Project [from_json(StructField(t,TimestampType,true), (timestampFormat,MM-dd), {"t": "02-29"}, Some(America/Los_Angeles)) AS from_json({"t": "02-29"})#x] ++- OneRowRelation + + +-- !query +select to_json(array('1', '2', '3')) +-- !query analysis +Project [to_json(array(1, 2, 3), Some(America/Los_Angeles)) AS to_json(array(1, 2, 3))#x] ++- OneRowRelation + + +-- !query +select to_json(array(array(1, 2, 3), array(4))) +-- !query analysis +Project [to_json(array(array(1, 2, 3), array(4)), Some(America/Los_Angeles)) AS to_json(array(array(1, 2, 3), array(4)))#x] ++- OneRowRelation + + +-- !query +select schema_of_json('{"c1":1}', map('primitivesAsString', 'true')) +-- !query analysis +Project [schema_of_json({"c1":1}, (primitivesAsString,true)) AS schema_of_json({"c1":1})#x] ++- OneRowRelation + + +-- !query +select schema_of_json('{"c1":01, "c2":0.1}', map('allowNumericLeadingZeros', 'true', 'prefersDecimal', 'true')) +-- !query analysis +Project [schema_of_json({"c1":01, "c2":0.1}, (allowNumericLeadingZeros,true), (prefersDecimal,true)) AS schema_of_json({"c1":01, "c2":0.1})#x] ++- OneRowRelation + + +-- !query +select schema_of_json(null) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "json", + "sqlExpr" : "\"schema_of_json(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "schema_of_json(null)" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a') +-- !query analysis +CreateViewCommand `jsonTable`, [(jsonField,None), (a,None)], SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a'), false, false, LocalTempView, true + +- Project [col1#x, col2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT schema_of_json(jsonField) FROM jsonTable +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"jsonField\"", + "inputName" : "json", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"schema_of_json(jsonField)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "schema_of_json(jsonField)" + } ] +} + + +-- !query +select json_array_length(null) +-- !query analysis +Project [json_array_length(null) AS json_array_length(NULL)#x] ++- OneRowRelation + + +-- !query +select json_array_length(2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"json_array_length(2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "json_array_length(2)" + } ] +} + + +-- !query +select json_array_length() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`json_array_length`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "json_array_length()" + } ] +} + + +-- !query +select json_array_length('') +-- !query analysis +Project [json_array_length() AS json_array_length()#x] ++- OneRowRelation + + +-- !query +select json_array_length('[]') +-- !query analysis +Project [json_array_length([]) AS json_array_length([])#x] ++- OneRowRelation + + +-- !query +select json_array_length('[1,2,3]') +-- !query analysis +Project [json_array_length([1,2,3]) AS json_array_length([1,2,3])#x] ++- OneRowRelation + + +-- !query +select json_array_length('[[1,2],[5,6,7]]') +-- !query analysis +Project [json_array_length([[1,2],[5,6,7]]) AS json_array_length([[1,2],[5,6,7]])#x] ++- OneRowRelation + + +-- !query +select json_array_length('[{"a":123},{"b":"hello"}]') +-- !query analysis +Project [json_array_length([{"a":123},{"b":"hello"}]) AS json_array_length([{"a":123},{"b":"hello"}])#x] ++- OneRowRelation + + +-- !query +select json_array_length('[1,2,3,[33,44],{"key":[2,3,4]}]') +-- !query analysis +Project [json_array_length([1,2,3,[33,44],{"key":[2,3,4]}]) AS json_array_length([1,2,3,[33,44],{"key":[2,3,4]}])#x] ++- OneRowRelation + + +-- !query +select json_array_length('{"key":"not a json array"}') +-- !query analysis +Project [json_array_length({"key":"not a json array"}) AS json_array_length({"key":"not a json array"})#x] ++- OneRowRelation + + +-- !query +select json_array_length('[1,2,3,4,5') +-- !query analysis +Project [json_array_length([1,2,3,4,5) AS json_array_length([1,2,3,4,5)#x] ++- OneRowRelation + + +-- !query +select json_object_keys() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`json_object_keys`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "json_object_keys()" + } ] +} + + +-- !query +select json_object_keys(null) +-- !query analysis +Project [json_object_keys(null) AS json_object_keys(NULL)#x] ++- OneRowRelation + + +-- !query +select json_object_keys(200) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"200\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"json_object_keys(200)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "json_object_keys(200)" + } ] +} + + +-- !query +select json_object_keys('') +-- !query analysis +Project [json_object_keys() AS json_object_keys()#x] ++- OneRowRelation + + +-- !query +select json_object_keys('{}') +-- !query analysis +Project [json_object_keys({}) AS json_object_keys({})#x] ++- OneRowRelation + + +-- !query +select json_object_keys('{"key": 1}') +-- !query analysis +Project [json_object_keys({"key": 1}) AS json_object_keys({"key": 1})#x] ++- OneRowRelation + + +-- !query +select json_object_keys('{"key": "value", "key2": 2}') +-- !query analysis +Project [json_object_keys({"key": "value", "key2": 2}) AS json_object_keys({"key": "value", "key2": 2})#x] ++- OneRowRelation + + +-- !query +select json_object_keys('{"arrayKey": [1, 2, 3]}') +-- !query analysis +Project [json_object_keys({"arrayKey": [1, 2, 3]}) AS json_object_keys({"arrayKey": [1, 2, 3]})#x] ++- OneRowRelation + + +-- !query +select json_object_keys('{"key":[1,2,3,{"key":"value"},[1,2,3]]}') +-- !query analysis +Project [json_object_keys({"key":[1,2,3,{"key":"value"},[1,2,3]]}) AS json_object_keys({"key":[1,2,3,{"key":"value"},[1,2,3]]})#x] ++- OneRowRelation + + +-- !query +select json_object_keys('{"f1":"abc","f2":{"f3":"a", "f4":"b"}}') +-- !query analysis +Project [json_object_keys({"f1":"abc","f2":{"f3":"a", "f4":"b"}}) AS json_object_keys({"f1":"abc","f2":{"f3":"a", "f4":"b"}})#x] ++- OneRowRelation + + +-- !query +select json_object_keys('{"k1": [1, 2, {"key": 5}], "k2": {"key2": [1, 2]}}') +-- !query analysis +Project [json_object_keys({"k1": [1, 2, {"key": 5}], "k2": {"key2": [1, 2]}}) AS json_object_keys({"k1": [1, 2, {"key": 5}], "k2": {"key2": [1, 2]}})#x] ++- OneRowRelation + + +-- !query +select json_object_keys('{[1,2]}') +-- !query analysis +Project [json_object_keys({[1,2]}) AS json_object_keys({[1,2]})#x] ++- OneRowRelation + + +-- !query +select json_object_keys('{"key": 45, "random_string"}') +-- !query analysis +Project [json_object_keys({"key": 45, "random_string"}) AS json_object_keys({"key": 45, "random_string"})#x] ++- OneRowRelation + + +-- !query +select json_object_keys('[1, 2, 3]') +-- !query analysis +Project [json_object_keys([1, 2, 3]) AS json_object_keys([1, 2, 3])#x] ++- OneRowRelation + + +-- !query +DROP VIEW IF EXISTS jsonTable +-- !query analysis +DropTempViewCommand jsonTable diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/like-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/like-all.sql.out new file mode 100644 index 0000000000000..f2ac4bd40cdc5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/like-all.sql.out @@ -0,0 +1,213 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW like_all_table AS SELECT * FROM (VALUES + ('google', '%oo%'), + ('facebook', '%oo%'), + ('linkedin', '%in')) + as t1(company, pat) +-- !query analysis +CreateViewCommand `like_all_table`, SELECT * FROM (VALUES + ('google', '%oo%'), + ('facebook', '%oo%'), + ('linkedin', '%in')) + as t1(company, pat), false, true, LocalTempView, true + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_all_table WHERE company LIKE ALL ('%oo%', '%go%') +-- !query analysis +Project [company#x] ++- Filter likeall(company#x, %oo%, %go%) + +- SubqueryAlias like_all_table + +- View (`like_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_all_table WHERE company LIKE ALL ('microsoft', '%yoo%') +-- !query analysis +Project [company#x] ++- Filter likeall(company#x, microsoft, %yoo%) + +- SubqueryAlias like_all_table + +- View (`like_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT + company, + CASE + WHEN company LIKE ALL ('%oo%', '%go%') THEN 'Y' + ELSE 'N' + END AS is_available, + CASE + WHEN company LIKE ALL ('%oo%', 'go%') OR company LIKE ALL ('%in', 'ms%') THEN 'Y' + ELSE 'N' + END AS mix +FROM like_all_table +-- !query analysis +Project [company#x, CASE WHEN likeall(company#x, %oo%, %go%) THEN Y ELSE N END AS is_available#x, CASE WHEN (likeall(company#x, %oo%, go%) OR likeall(company#x, %in, ms%)) THEN Y ELSE N END AS mix#x] ++- SubqueryAlias like_all_table + +- View (`like_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_all_table WHERE company LIKE ALL ('%oo%', pat) +-- !query analysis +Project [company#x] ++- Filter (company#x LIKE %oo% AND company#x LIKE pat#x) + +- SubqueryAlias like_all_table + +- View (`like_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_all_table WHERE company NOT LIKE ALL ('%oo%', '%in', 'fa%') +-- !query analysis +Project [company#x] ++- Filter notlikeall(company#x, %oo%, %in, fa%) + +- SubqueryAlias like_all_table + +- View (`like_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_all_table WHERE company NOT LIKE ALL ('microsoft', '%yoo%') +-- !query analysis +Project [company#x] ++- Filter notlikeall(company#x, microsoft, %yoo%) + +- SubqueryAlias like_all_table + +- View (`like_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_all_table WHERE company NOT LIKE ALL ('%oo%', 'fa%') +-- !query analysis +Project [company#x] ++- Filter notlikeall(company#x, %oo%, fa%) + +- SubqueryAlias like_all_table + +- View (`like_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_all_table WHERE NOT company LIKE ALL ('%oo%', 'fa%') +-- !query analysis +Project [company#x] ++- Filter NOT likeall(company#x, %oo%, fa%) + +- SubqueryAlias like_all_table + +- View (`like_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_all_table WHERE company LIKE ALL ('%oo%', NULL) +-- !query analysis +Project [company#x] ++- Filter (company#x LIKE %oo% AND company#x LIKE cast(null as string)) + +- SubqueryAlias like_all_table + +- View (`like_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_all_table WHERE company NOT LIKE ALL ('%oo%', NULL) +-- !query analysis +Project [company#x] ++- Filter (NOT company#x LIKE %oo% AND NOT company#x LIKE cast(null as string)) + +- SubqueryAlias like_all_table + +- View (`like_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_all_table WHERE company LIKE ALL (NULL, NULL) +-- !query analysis +Project [company#x] ++- Filter (company#x LIKE cast(null as string) AND company#x LIKE cast(null as string)) + +- SubqueryAlias like_all_table + +- View (`like_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_all_table WHERE company NOT LIKE ALL (NULL, NULL) +-- !query analysis +Project [company#x] ++- Filter (NOT company#x LIKE cast(null as string) AND NOT company#x LIKE cast(null as string)) + +- SubqueryAlias like_all_table + +- View (`like_all_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_all_table WHERE company LIKE ALL () +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Expected something between '(' and ')'." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 50, + "stopIndex" : 60, + "fragment" : "LIKE ALL ()" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/like-any.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/like-any.sql.out new file mode 100644 index 0000000000000..58986a38aaf7c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/like-any.sql.out @@ -0,0 +1,213 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW like_any_table AS SELECT * FROM (VALUES + ('google', '%oo%'), + ('facebook', '%oo%'), + ('linkedin', '%in')) + as t1(company, pat) +-- !query analysis +CreateViewCommand `like_any_table`, SELECT * FROM (VALUES + ('google', '%oo%'), + ('facebook', '%oo%'), + ('linkedin', '%in')) + as t1(company, pat), false, true, LocalTempView, true + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_any_table WHERE company LIKE ANY ('%oo%', '%in', 'fa%') +-- !query analysis +Project [company#x] ++- Filter likeany(company#x, %oo%, %in, fa%) + +- SubqueryAlias like_any_table + +- View (`like_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_any_table WHERE company LIKE ANY ('microsoft', '%yoo%') +-- !query analysis +Project [company#x] ++- Filter likeany(company#x, microsoft, %yoo%) + +- SubqueryAlias like_any_table + +- View (`like_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select + company, + CASE + WHEN company LIKE ANY ('%oo%', '%in', 'fa%') THEN 'Y' + ELSE 'N' + END AS is_available, + CASE + WHEN company LIKE ANY ('%oo%', 'fa%') OR company LIKE ANY ('%in', 'ms%') THEN 'Y' + ELSE 'N' + END AS mix +FROM like_any_table +-- !query analysis +Project [company#x, CASE WHEN likeany(company#x, %oo%, %in, fa%) THEN Y ELSE N END AS is_available#x, CASE WHEN (likeany(company#x, %oo%, fa%) OR likeany(company#x, %in, ms%)) THEN Y ELSE N END AS mix#x] ++- SubqueryAlias like_any_table + +- View (`like_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_any_table WHERE company LIKE ANY ('%zz%', pat) +-- !query analysis +Project [company#x] ++- Filter (company#x LIKE %zz% OR company#x LIKE pat#x) + +- SubqueryAlias like_any_table + +- View (`like_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_any_table WHERE company NOT LIKE ANY ('%oo%', '%in', 'fa%') +-- !query analysis +Project [company#x] ++- Filter notlikeany(company#x, %oo%, %in, fa%) + +- SubqueryAlias like_any_table + +- View (`like_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_any_table WHERE company NOT LIKE ANY ('microsoft', '%yoo%') +-- !query analysis +Project [company#x] ++- Filter notlikeany(company#x, microsoft, %yoo%) + +- SubqueryAlias like_any_table + +- View (`like_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_any_table WHERE company NOT LIKE ANY ('%oo%', 'fa%') +-- !query analysis +Project [company#x] ++- Filter notlikeany(company#x, %oo%, fa%) + +- SubqueryAlias like_any_table + +- View (`like_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_any_table WHERE NOT company LIKE ANY ('%oo%', 'fa%') +-- !query analysis +Project [company#x] ++- Filter NOT likeany(company#x, %oo%, fa%) + +- SubqueryAlias like_any_table + +- View (`like_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_any_table WHERE company LIKE ANY ('%oo%', NULL) +-- !query analysis +Project [company#x] ++- Filter (company#x LIKE %oo% OR company#x LIKE cast(null as string)) + +- SubqueryAlias like_any_table + +- View (`like_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_any_table WHERE company NOT LIKE ANY ('%oo%', NULL) +-- !query analysis +Project [company#x] ++- Filter (NOT company#x LIKE %oo% OR NOT company#x LIKE cast(null as string)) + +- SubqueryAlias like_any_table + +- View (`like_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_any_table WHERE company LIKE ANY (NULL, NULL) +-- !query analysis +Project [company#x] ++- Filter (company#x LIKE cast(null as string) OR company#x LIKE cast(null as string)) + +- SubqueryAlias like_any_table + +- View (`like_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_any_table WHERE company NOT LIKE ANY (NULL, NULL) +-- !query analysis +Project [company#x] ++- Filter (NOT company#x LIKE cast(null as string) OR NOT company#x LIKE cast(null as string)) + +- SubqueryAlias like_any_table + +- View (`like_any_table`, [company#x,pat#x]) + +- Project [cast(company#x as string) AS company#x, cast(pat#x as string) AS pat#x] + +- Project [company#x, pat#x] + +- SubqueryAlias t1 + +- Project [col1#x AS company#x, col2#x AS pat#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT company FROM like_any_table WHERE company LIKE ANY () +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Expected something between '(' and ')'." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 50, + "stopIndex" : 60, + "fragment" : "LIKE ANY ()" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out new file mode 100644 index 0000000000000..c38e46966054a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out @@ -0,0 +1,193 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT * FROM testdata LIMIT 2 +-- !query analysis +GlobalLimit 2 ++- LocalLimit 2 + +- Project [key#x, value#x] + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +SELECT * FROM arraydata LIMIT 2 +-- !query analysis +GlobalLimit 2 ++- LocalLimit 2 + +- Project [arraycol#x, nestedarraycol#x] + +- SubqueryAlias spark_catalog.default.arraydata + +- Relation spark_catalog.default.arraydata[arraycol#x,nestedarraycol#x] parquet + + +-- !query +SELECT * FROM mapdata LIMIT 2 +-- !query analysis +GlobalLimit 2 ++- LocalLimit 2 + +- Project [mapcol#x] + +- SubqueryAlias spark_catalog.default.mapdata + +- Relation spark_catalog.default.mapdata[mapcol#x] parquet + + +-- !query +SELECT * FROM testdata LIMIT 2 + 1 +-- !query analysis +GlobalLimit (2 + 1) ++- LocalLimit (2 + 1) + +- Project [key#x, value#x] + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +SELECT * FROM testdata LIMIT CAST(1 AS int) +-- !query analysis +GlobalLimit cast(1 as int) ++- LocalLimit cast(1 as int) + +- Project [key#x, value#x] + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +SELECT * FROM testdata LIMIT -1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2403", + "messageParameters" : { + "name" : "limit", + "v" : "-1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 31, + "fragment" : "-1" + } ] +} + + +-- !query +SELECT * FROM testData TABLESAMPLE (-1 ROWS) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2403", + "messageParameters" : { + "name" : "limit", + "v" : "-1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 37, + "stopIndex" : 38, + "fragment" : "-1" + } ] +} + + +-- !query +SELECT * FROM testdata LIMIT CAST(1 AS INT) +-- !query analysis +GlobalLimit cast(1 as int) ++- LocalLimit cast(1 as int) + +- Project [key#x, value#x] + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +SELECT * FROM testdata LIMIT CAST(NULL AS INT) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2402", + "messageParameters" : { + "limitExpr" : "CAST(NULL AS INT)", + "name" : "limit" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 46, + "fragment" : "CAST(NULL AS INT)" + } ] +} + + +-- !query +SELECT * FROM testdata LIMIT key > 3 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2400", + "messageParameters" : { + "limitExpr" : "(spark_catalog.default.testdata.key > 3)", + "name" : "limit" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 36, + "fragment" : "key > 3" + } ] +} + + +-- !query +SELECT * FROM testdata LIMIT true +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2401", + "messageParameters" : { + "dataType" : "boolean", + "name" : "limit" + } +} + + +-- !query +SELECT * FROM testdata LIMIT 'a' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2401", + "messageParameters" : { + "dataType" : "string", + "name" : "limit" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 32, + "fragment" : "'a'" + } ] +} + + +-- !query +SELECT * FROM (SELECT * FROM range(10) LIMIT 5) WHERE id > 3 +-- !query analysis +Project [id#xL] ++- Filter (id#xL > cast(3 as bigint)) + +- SubqueryAlias __auto_generated_subquery_name + +- GlobalLimit 5 + +- LocalLimit 5 + +- Project [id#xL] + +- Range (0, 10, step=1, splits=None) + + +-- !query +SELECT * FROM testdata WHERE key < 3 LIMIT ALL +-- !query analysis +Project [key#x, value#x] ++- Filter (key#x < 3) + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/linear-regression.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/linear-regression.sql.out new file mode 100644 index 0000000000000..7c91139921b58 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/linear-regression.sql.out @@ -0,0 +1,409 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testRegression AS SELECT * FROM VALUES +(1, 10, null), (2, 10, 11), (2, 20, 22), (2, 25, null), (2, 30, 35) +AS testRegression(k, y, x) +-- !query analysis +CreateViewCommand `testRegression`, SELECT * FROM VALUES +(1, 10, null), (2, 10, 11), (2, 20, 22), (2, 25, null), (2, 30, 35) +AS testRegression(k, y, x), false, true, LocalTempView, true + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_count(y, x) FROM testRegression +-- !query analysis +Aggregate [regr_count(y#x, x#x) AS regr_count(y, x)#xL] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_count(y, x) FROM testRegression WHERE x IS NOT NULL +-- !query analysis +Aggregate [regr_count(y#x, x#x) AS regr_count(y, x)#xL] ++- Filter isnotnull(x#x) + +- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, count(*), regr_count(y, x) FROM testRegression GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, count(1) AS count(1)#xL, regr_count(y#x, x#x) AS regr_count(y, x)#xL] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, count(*) FILTER (WHERE x IS NOT NULL), regr_count(y, x) FROM testRegression GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, count(1) FILTER (WHERE isnotnull(x#x)) AS count(1) FILTER (WHERE (x IS NOT NULL))#xL, regr_count(y#x, x#x) AS regr_count(y, x)#xL] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_r2(y, x) FROM testRegression +-- !query analysis +Aggregate [regr_r2(cast(y#x as double), cast(x#x as double)) AS regr_r2(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_r2(y, x) FROM testRegression WHERE x IS NOT NULL +-- !query analysis +Aggregate [regr_r2(cast(y#x as double), cast(x#x as double)) AS regr_r2(y, x)#x] ++- Filter isnotnull(x#x) + +- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, corr(y, x), regr_r2(y, x) FROM testRegression GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, corr(cast(y#x as double), cast(x#x as double)) AS corr(y, x)#x, regr_r2(cast(y#x as double), cast(x#x as double)) AS regr_r2(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, corr(y, x) FILTER (WHERE x IS NOT NULL), regr_r2(y, x) FROM testRegression GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, corr(cast(y#x as double), cast(x#x as double)) FILTER (WHERE isnotnull(x#x)) AS corr(y, x) FILTER (WHERE (x IS NOT NULL))#x, regr_r2(cast(y#x as double), cast(x#x as double)) AS regr_r2(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression +-- !query analysis +Aggregate [regr_avgx(y#x, x#x) AS regr_avgx(y, x)#x, regr_avgy(y#x, x#x) AS regr_avgy(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query analysis +Aggregate [regr_avgx(y#x, x#x) AS regr_avgx(y, x)#x, regr_avgy(y#x, x#x) AS regr_avgy(y, x)#x] ++- Filter (isnotnull(x#x) AND isnotnull(y#x)) + +- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, avg(x), avg(y), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, avg(x#x) AS avg(x)#x, avg(y#x) AS avg(y)#x, regr_avgx(y#x, x#x) AS regr_avgx(y, x)#x, regr_avgy(y#x, x#x) AS regr_avgy(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, avg(x) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), avg(y) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, avg(x#x) FILTER (WHERE (isnotnull(x#x) AND isnotnull(y#x))) AS avg(x) FILTER (WHERE ((x IS NOT NULL) AND (y IS NOT NULL)))#x, avg(y#x) FILTER (WHERE (isnotnull(x#x) AND isnotnull(y#x))) AS avg(y) FILTER (WHERE ((x IS NOT NULL) AND (y IS NOT NULL)))#x, regr_avgx(y#x, x#x) AS regr_avgx(y, x)#x, regr_avgy(y#x, x#x) AS regr_avgy(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_sxx(y, x) FROM testRegression +-- !query analysis +Aggregate [regr_sxx(cast(y#x as double), cast(x#x as double)) AS regr_sxx(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query analysis +Aggregate [regr_sxx(cast(y#x as double), cast(x#x as double)) AS regr_sxx(y, x)#x] ++- Filter (isnotnull(x#x) AND isnotnull(y#x)) + +- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, regr_sxx(y, x) FROM testRegression GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, regr_sxx(cast(y#x as double), cast(x#x as double)) AS regr_sxx(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, regr_sxx(cast(y#x as double), cast(x#x as double)) AS regr_sxx(y, x)#x] ++- Filter (isnotnull(x#x) AND isnotnull(y#x)) + +- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_sxy(y, x) FROM testRegression +-- !query analysis +Aggregate [regr_sxy(cast(y#x as double), cast(x#x as double)) AS regr_sxy(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query analysis +Aggregate [regr_sxy(cast(y#x as double), cast(x#x as double)) AS regr_sxy(y, x)#x] ++- Filter (isnotnull(x#x) AND isnotnull(y#x)) + +- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, regr_sxy(y, x) FROM testRegression GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, regr_sxy(cast(y#x as double), cast(x#x as double)) AS regr_sxy(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, regr_sxy(cast(y#x as double), cast(x#x as double)) AS regr_sxy(y, x)#x] ++- Filter (isnotnull(x#x) AND isnotnull(y#x)) + +- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_syy(y, x) FROM testRegression +-- !query analysis +Aggregate [regr_syy(cast(y#x as double), cast(x#x as double)) AS regr_syy(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query analysis +Aggregate [regr_syy(cast(y#x as double), cast(x#x as double)) AS regr_syy(y, x)#x] ++- Filter (isnotnull(x#x) AND isnotnull(y#x)) + +- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, regr_syy(y, x) FROM testRegression GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, regr_syy(cast(y#x as double), cast(x#x as double)) AS regr_syy(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, regr_syy(cast(y#x as double), cast(x#x as double)) AS regr_syy(y, x)#x] ++- Filter (isnotnull(x#x) AND isnotnull(y#x)) + +- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_slope(y, x) FROM testRegression +-- !query analysis +Aggregate [regr_slope(cast(y#x as double), cast(x#x as double)) AS regr_slope(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query analysis +Aggregate [regr_slope(cast(y#x as double), cast(x#x as double)) AS regr_slope(y, x)#x] ++- Filter (isnotnull(x#x) AND isnotnull(y#x)) + +- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, regr_slope(y, x) FROM testRegression GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, regr_slope(cast(y#x as double), cast(x#x as double)) AS regr_slope(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, regr_slope(cast(y#x as double), cast(x#x as double)) AS regr_slope(y, x)#x] ++- Filter (isnotnull(x#x) AND isnotnull(y#x)) + +- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_intercept(y, x) FROM testRegression +-- !query analysis +Aggregate [regr_intercept(cast(y#x as double), cast(x#x as double)) AS regr_intercept(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query analysis +Aggregate [regr_intercept(cast(y#x as double), cast(x#x as double)) AS regr_intercept(y, x)#x] ++- Filter (isnotnull(x#x) AND isnotnull(y#x)) + +- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, regr_intercept(y, x) FROM testRegression GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, regr_intercept(cast(y#x as double), cast(x#x as double)) AS regr_intercept(y, x)#x] ++- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] + + +-- !query +SELECT k, regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query analysis +Aggregate [k#x], [k#x, regr_intercept(cast(y#x as double), cast(x#x as double)) AS regr_intercept(y, x)#x] ++- Filter (isnotnull(x#x) AND isnotnull(y#x)) + +- SubqueryAlias testregression + +- View (`testRegression`, [k#x,y#x,x#x]) + +- Project [cast(k#x as int) AS k#x, cast(y#x as int) AS y#x, cast(x#x as int) AS x#x] + +- Project [k#x, y#x, x#x] + +- SubqueryAlias testRegression + +- LocalRelation [k#x, y#x, x#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/literals.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/literals.sql.out new file mode 100644 index 0000000000000..9e908ad798e41 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/literals.sql.out @@ -0,0 +1,694 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select null, Null, nUll +-- !query analysis +Project [null AS NULL#x, null AS NULL#x, null AS NULL#x] ++- OneRowRelation + + +-- !query +select true, tRue, false, fALse +-- !query analysis +Project [true AS true#x, true AS true#x, false AS false#x, false AS false#x] ++- OneRowRelation + + +-- !query +select 1Y +-- !query analysis +Project [1 AS 1#x] ++- OneRowRelation + + +-- !query +select 127Y, -128Y +-- !query analysis +Project [127 AS 127#x, -128 AS -128#x] ++- OneRowRelation + + +-- !query +select 128Y +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "127", + "minValue" : "-128", + "rawStrippedQualifier" : "128", + "typeName" : "tinyint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 11, + "fragment" : "128Y" + } ] +} + + +-- !query +select 1S +-- !query analysis +Project [1 AS 1#x] ++- OneRowRelation + + +-- !query +select 32767S, -32768S +-- !query analysis +Project [32767 AS 32767#x, -32768 AS -32768#x] ++- OneRowRelation + + +-- !query +select 32768S +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "32767", + "minValue" : "-32768", + "rawStrippedQualifier" : "32768", + "typeName" : "smallint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : "32768S" + } ] +} + + +-- !query +select 1L, 2147483648L +-- !query analysis +Project [1 AS 1#xL, 2147483648 AS 2147483648#xL] ++- OneRowRelation + + +-- !query +select 9223372036854775807L, -9223372036854775808L +-- !query analysis +Project [9223372036854775807 AS 9223372036854775807#xL, -9223372036854775808 AS -9223372036854775808#xL] ++- OneRowRelation + + +-- !query +select 9223372036854775808L +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "9223372036854775807", + "minValue" : "-9223372036854775808", + "rawStrippedQualifier" : "9223372036854775808", + "typeName" : "bigint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "9223372036854775808L" + } ] +} + + +-- !query +select 1, -1 +-- !query analysis +Project [1 AS 1#x, -1 AS -1#x] ++- OneRowRelation + + +-- !query +select 2147483647, -2147483648 +-- !query analysis +Project [2147483647 AS 2147483647#x, -2147483648 AS -2147483648#x] ++- OneRowRelation + + +-- !query +select 9223372036854775807, -9223372036854775808 +-- !query analysis +Project [9223372036854775807 AS 9223372036854775807#xL, -9223372036854775808 AS -9223372036854775808#xL] ++- OneRowRelation + + +-- !query +select 9223372036854775808, -9223372036854775809 +-- !query analysis +Project [9223372036854775808 AS 9223372036854775808#x, -9223372036854775809 AS -9223372036854775809#x] ++- OneRowRelation + + +-- !query +select 1234567890123456789012345678901234567890 +-- !query analysis +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", + "sqlState" : "22003", + "messageParameters" : { + "maxPrecision" : "38", + "precision" : "40" + } +} + + +-- !query +select 1234567890123456789012345678901234567890.0 +-- !query analysis +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", + "sqlState" : "22003", + "messageParameters" : { + "maxPrecision" : "38", + "precision" : "41" + } +} + + +-- !query +select 1F, 1.2F, .10f, 0.10f +-- !query analysis +Project [1.0 AS 1.0#x, 1.2 AS 1.2#x, 0.1 AS 0.1#x, 0.1 AS 0.1#x] ++- OneRowRelation + + +-- !query +select -1F, -1.2F, -.10F, -0.10F +-- !query analysis +Project [-1.0 AS -1.0#x, -1.2 AS -1.2#x, -0.1 AS -0.1#x, -0.1 AS -0.1#x] ++- OneRowRelation + + +-- !query +select -3.4028235E39f +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "3.4028234663852886E+38", + "minValue" : "-3.4028234663852886E+38", + "rawStrippedQualifier" : "-3.4028235E39", + "typeName" : "float" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "-3.4028235E39f" + } ] +} + + +-- !query +select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 +-- !query analysis +Project [1.0 AS 1.0#x, 1.2 AS 1.2#x, 1.0E10 AS 1.0E10#x, 150000.0 AS 150000.0#x, 0.1 AS 0.1#x, 0.1 AS 0.1#x, 10000.0 AS 10000.0#x, 90.0 AS 90.0#x, 90.0 AS 90.0#x, 90.0 AS 90.0#x, 90.0 AS 90.0#x] ++- OneRowRelation + + +-- !query +select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5 +-- !query analysis +Project [-1.0 AS -1.0#x, -1.2 AS -1.2#x, -1.0E10 AS -1.0E10#x, -150000.0 AS -150000.0#x, -0.1 AS -0.1#x, -0.1 AS -0.1#x, -10000.0 AS -10000.0#x] ++- OneRowRelation + + +-- !query +select .e3 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'.'", + "hint" : "" + } +} + + +-- !query +select 1E309, -1E309 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0023", + "messageParameters" : { + "maxValue" : "1.7976931348623157E+308", + "minValue" : "-1.7976931348623157E+308", + "rawStrippedQualifier" : "1E309", + "typeName" : "double" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "1E309" + } ] +} + + +-- !query +select 0.3, -0.8, .5, -.18, 0.1111, .1111 +-- !query analysis +Project [0.3 AS 0.3#x, -0.8 AS -0.8#x, 0.5 AS 0.5#x, -0.18 AS -0.18#x, 0.1111 AS 0.1111#x, 0.1111 AS 0.1111#x] ++- OneRowRelation + + +-- !query +select 0.3 F, 0.4 D, 0.5 BD +-- !query analysis +Project [0.3 AS F#x, 0.4 AS D#x, 0.5 AS BD#x] ++- OneRowRelation + + +-- !query +select 123456789012345678901234567890123456789e10d, 123456789012345678901234567890123456789.1e10d +-- !query analysis +Project [1.2345678901234568E48 AS 1.2345678901234568E48#x, 1.2345678901234568E48 AS 1.2345678901234568E48#x] ++- OneRowRelation + + +-- !query +select "Hello Peter!", 'hello lee!' +-- !query analysis +Project [Hello Peter! AS Hello Peter!#x, hello lee! AS hello lee!#x] ++- OneRowRelation + + +-- !query +select 'hello' 'world', 'hello' " " 'lee' +-- !query analysis +Project [helloworld AS helloworld#x, hello lee AS hello lee#x] ++- OneRowRelation + + +-- !query +select "hello 'peter'" +-- !query analysis +Project [hello 'peter' AS hello 'peter'#x] ++- OneRowRelation + + +-- !query +select 'pattern%', 'no-pattern\%', 'pattern\\%', 'pattern\\\%' +-- !query analysis +Project [pattern% AS pattern%#x, no-pattern\% AS no-pattern\%#x, pattern\% AS pattern\%#x, pattern\\% AS pattern\\%#x] ++- OneRowRelation + + +-- !query +select '\'', '"', '\n', '\r', '\t', 'Z' +-- !query analysis +Project [' AS '#x, " AS "#x, + AS +#x, AS #x, AS #x, Z AS Z#x] ++- OneRowRelation + + +-- !query +select '\110\145\154\154\157\041' +-- !query analysis +Project [Hello! AS Hello!#x] ++- OneRowRelation + + +-- !query +select '\u0057\u006F\u0072\u006C\u0064\u0020\u003A\u0029' +-- !query analysis +Project [World :) AS World :)#x] ++- OneRowRelation + + +-- !query +select dAte '2016-03-12' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date 'mar 11 2016' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'mar 11 2016'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date 'mar 11 2016'" + } ] +} + + +-- !query +select tImEstAmp '2016-03-11 20:54:00.000' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp '2016-33-11 20:54:00.000' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2016-33-11 20:54:00.000'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "timestamp '2016-33-11 20:54:00.000'" + } ] +} + + +-- !query +select GEO '(10,-6)' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_TYPED_LITERAL", + "sqlState" : "0A000", + "messageParameters" : { + "supportedTypes" : "\"DATE\", \"TIMESTAMP_NTZ\", \"TIMESTAMP_LTZ\", \"TIMESTAMP\", \"INTERVAL\", \"X\"", + "unsupportedType" : "\"GEO\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "GEO '(10,-6)'" + } ] +} + + +-- !query +select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD +-- !query analysis +Project [90912830918230182310293801923652346786 AS 90912830918230182310293801923652346786#x, 1.230E-26 AS 1.230E-26#x, 123.08 AS 123.08#x] ++- OneRowRelation + + +-- !query +select 1.20E-38BD +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0061", + "messageParameters" : { + "msg" : "[DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION] Decimal precision 40 exceeds max precision 38." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "1.20E-38BD" + } ] +} + + +-- !query +select x'2379ACFe' +-- !query analysis +Project [0x2379ACFE AS X'2379ACFE'#x] ++- OneRowRelation + + +-- !query +select X'XuZ' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'XuZ'", + "valueType" : "\"X\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : "X'XuZ'" + } ] +} + + +-- !query +SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 +-- !query analysis +Project [3.14 AS 3.14#x, -3.14 AS -3.14#x, 3.14E8 AS 3.14E8#x, 3.14E-8 AS 3.14E-8#x, -3.14E8 AS -3.14E8#x, -3.14E-8 AS -3.14E-8#x, 3.14E8 AS 3.14E8#x, 3.14E8 AS 3.14E8#x, 3.14E-8 AS 3.14E-8#x] ++- OneRowRelation + + +-- !query +select +date '1999-01-01' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"DATE '1999-01-01'\"", + "inputType" : "\"DATE\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ DATE '1999-01-01')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "+date '1999-01-01'" + } ] +} + + +-- !query +select +timestamp '1999-01-01' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"TIMESTAMP '1999-01-01 00:00:00'\"", + "inputType" : "\"TIMESTAMP\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ TIMESTAMP '1999-01-01 00:00:00')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "+timestamp '1999-01-01'" + } ] +} + + +-- !query +select +interval '1 day' +-- !query analysis +Project [positive(INTERVAL '1' DAY) AS (+ INTERVAL '1' DAY)#x] ++- OneRowRelation + + +-- !query +select +map(1, 2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"map(1, 2)\"", + "inputType" : "\"MAP\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ map(1, 2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "+map(1, 2)" + } ] +} + + +-- !query +select +array(1,2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"array(1, 2)\"", + "inputType" : "\"ARRAY\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ array(1, 2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "+array(1,2)" + } ] +} + + +-- !query +select +named_struct('a', 1, 'b', 'spark') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"named_struct(a, 1, b, spark)\"", + "inputType" : "\"STRUCT\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ named_struct(a, 1, b, spark))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "+named_struct('a', 1, 'b', 'spark')" + } ] +} + + +-- !query +select +X'1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"X'01'\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ X'01')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "+X'1'" + } ] +} + + +-- !query +select -date '1999-01-01' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"DATE '1999-01-01'\"", + "inputType" : "\"DATE\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(- DATE '1999-01-01')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "-date '1999-01-01'" + } ] +} + + +-- !query +select -timestamp '1999-01-01' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"TIMESTAMP '1999-01-01 00:00:00'\"", + "inputType" : "\"TIMESTAMP\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(- TIMESTAMP '1999-01-01 00:00:00')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "-timestamp '1999-01-01'" + } ] +} + + +-- !query +select -x'2379ACFe' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"X'2379ACFE'\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "1", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(- X'2379ACFE')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "-x'2379ACFe'" + } ] +} 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 new file mode 100644 index 0000000000000..79b12dc689114 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/map.sql.out @@ -0,0 +1,103 @@ +-- Automatically generated by SQLQueryTestSuite +-- !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] ++- OneRowRelation + + +-- !query +select map(1, 'a', 2, 'b')[5] +-- !query analysis +Project [map(1, a, 2, b)[5] AS map(1, a, 2, b)[5]#x] ++- OneRowRelation + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 5) +-- !query analysis +Project [map_contains_key(map(1, a, 2, b), 5) AS map_contains_key(map(1, a, 2, b), 5)#x] ++- OneRowRelation + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 1) +-- !query analysis +Project [map_contains_key(map(1, a, 2, b), 1) AS map_contains_key(map(1, a, 2, b), 1)#x] ++- OneRowRelation + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 5.0) +-- !query analysis +Project [map_contains_key(cast(map(1, a, 2, b) as map), cast(5.0 as decimal(11,1))) AS map_contains_key(map(1, a, 2, b), 5.0)#x] ++- OneRowRelation + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 1.0) +-- !query analysis +Project [map_contains_key(cast(map(1, a, 2, b) as map), cast(1.0 as decimal(11,1))) AS map_contains_key(map(1, a, 2, b), 1.0)#x] ++- OneRowRelation + + +-- !query +select map_contains_key(map(1.0, 'a', 2, 'b'), 5) +-- !query analysis +Project [map_contains_key(map(cast(1.0 as decimal(11,1)), a, cast(2 as decimal(11,1)), b), cast(5 as decimal(11,1))) AS map_contains_key(map(1.0, a, 2, b), 5)#x] ++- OneRowRelation + + +-- !query +select map_contains_key(map(1.0, 'a', 2, 'b'), 1) +-- !query analysis +Project [map_contains_key(map(cast(1.0 as decimal(11,1)), a, cast(2 as decimal(11,1)), b), cast(1 as decimal(11,1))) AS map_contains_key(map(1.0, a, 2, b), 1)#x] ++- OneRowRelation + + +-- !query +select map_contains_key(map('1', 'a', '2', 'b'), 1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "\"MAP\"", + "functionName" : "`map_contains_key`", + "leftType" : "\"MAP\"", + "rightType" : "\"INT\"", + "sqlExpr" : "\"map_contains_key(map(1, a, 2, b), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "map_contains_key(map('1', 'a', '2', 'b'), 1)" + } ] +} + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), '1') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "\"MAP\"", + "functionName" : "`map_contains_key`", + "leftType" : "\"MAP\"", + "rightType" : "\"STRING\"", + "sqlExpr" : "\"map_contains_key(map(1, a, 2, b), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "map_contains_key(map(1, 'a', 2, 'b'), '1')" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/mask-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/mask-functions.sql.out new file mode 100644 index 0000000000000..1abe00ad4709f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/mask-functions.sql.out @@ -0,0 +1,512 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT mask('AbCD123-@$#') +-- !query analysis +Project [mask(AbCD123-@$#, X, x, n, null) AS mask(AbCD123-@$#, X, x, n, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT mask('AbCD123-@$#', 'Q') +-- !query analysis +Project [mask(AbCD123-@$#, Q, x, n, null) AS mask(AbCD123-@$#, Q, x, n, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT mask('AbCD123-@$#', 'Q', 'q') +-- !query analysis +Project [mask(AbCD123-@$#, Q, q, n, null) AS mask(AbCD123-@$#, Q, q, n, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT mask('AbCD123-@$#', 'Q', 'q', 'd') +-- !query analysis +Project [mask(AbCD123-@$#, Q, q, d, null) AS mask(AbCD123-@$#, Q, q, d, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT mask('AbCD123-@$#', 'Q', 'q', 'd', 'o') +-- !query analysis +Project [mask(AbCD123-@$#, Q, q, d, o) AS mask(AbCD123-@$#, Q, q, d, o)#x] ++- OneRowRelation + + +-- !query +SELECT mask('AbCD123-@$#', 'Qa', 'qa', 'da', 'oa') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "upperChar", + "sqlExpr" : "\"mask(AbCD123-@$#, Qa, qa, da, oa)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "mask('AbCD123-@$#', 'Qa', 'qa', 'da', 'oa')" + } ] +} + + +-- !query +SELECT mask('AbCD123-@$#', NULL, 'q', 'd', 'o') +-- !query analysis +Project [mask(AbCD123-@$#, null, q, d, o) AS mask(AbCD123-@$#, NULL, q, d, o)#x] ++- OneRowRelation + + +-- !query +SELECT mask('AbCD123-@$#', NULL, NULL, 'd', 'o') +-- !query analysis +Project [mask(AbCD123-@$#, null, null, d, o) AS mask(AbCD123-@$#, NULL, NULL, d, o)#x] ++- OneRowRelation + + +-- !query +SELECT mask('AbCD123-@$#', NULL, NULL, NULL, 'o') +-- !query analysis +Project [mask(AbCD123-@$#, null, null, null, o) AS mask(AbCD123-@$#, NULL, NULL, NULL, o)#x] ++- OneRowRelation + + +-- !query +SELECT mask('AbCD123-@$#', NULL, NULL, NULL, NULL) +-- !query analysis +Project [mask(AbCD123-@$#, null, null, null, null) AS mask(AbCD123-@$#, NULL, NULL, NULL, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT mask(NULL) +-- !query analysis +Project [mask(null, X, x, n, null) AS mask(NULL, X, x, n, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT mask(NULL, NULL, 'q', 'd', 'o') +-- !query analysis +Project [mask(null, null, q, d, o) AS mask(NULL, NULL, q, d, o)#x] ++- OneRowRelation + + +-- !query +SELECT mask(NULL, NULL, NULL, 'd', 'o') +-- !query analysis +Project [mask(null, null, null, d, o) AS mask(NULL, NULL, NULL, d, o)#x] ++- OneRowRelation + + +-- !query +SELECT mask(NULL, NULL, NULL, NULL, 'o') +-- !query analysis +Project [mask(null, null, null, null, o) AS mask(NULL, NULL, NULL, NULL, o)#x] ++- OneRowRelation + + +-- !query +SELECT mask('AbCD123-@$#', NULL, NULL, NULL, NULL) +-- !query analysis +Project [mask(AbCD123-@$#, null, null, null, null) AS mask(AbCD123-@$#, NULL, NULL, NULL, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT mask(c1) from values ('AbCD123-@$#') as tab(c1) +-- !query analysis +Project [mask(c1#x, X, x, n, null) AS mask(c1, X, x, n, NULL)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, 'Q') from values ('AbCD123-@$#') as tab(c1) +-- !query analysis +Project [mask(c1#x, Q, x, n, null) AS mask(c1, Q, x, n, NULL)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, 'Q', 'q')from values ('AbCD123-@$#') as tab(c1) +-- !query analysis +Project [mask(c1#x, Q, q, n, null) AS mask(c1, Q, q, n, NULL)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, 'Q', 'q', 'd') from values ('AbCD123-@$#') as tab(c1) +-- !query analysis +Project [mask(c1#x, Q, q, d, null) AS mask(c1, Q, q, d, NULL)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, 'Q', 'q', 'd', 'o') from values ('AbCD123-@$#') as tab(c1) +-- !query analysis +Project [mask(c1#x, Q, q, d, o) AS mask(c1, Q, q, d, o)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, NULL, 'q', 'd', 'o') from values ('AbCD123-@$#') as tab(c1) +-- !query analysis +Project [mask(c1#x, null, q, d, o) AS mask(c1, NULL, q, d, o)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, NULL, NULL, 'd', 'o') from values ('AbCD123-@$#') as tab(c1) +-- !query analysis +Project [mask(c1#x, null, null, d, o) AS mask(c1, NULL, NULL, d, o)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, NULL, NULL, NULL, 'o') from values ('AbCD123-@$#') as tab(c1) +-- !query analysis +Project [mask(c1#x, null, null, null, o) AS mask(c1, NULL, NULL, NULL, o)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, NULL, NULL, NULL, NULL) from values ('AbCD123-@$#') as tab(c1) +-- !query analysis +Project [mask(c1#x, null, null, null, null) AS mask(c1, NULL, NULL, NULL, NULL)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, NULL, 'q', 'd', 'o') from values ('AbCD123-@$#') as tab(c1) +-- !query analysis +Project [mask(c1#x, null, q, d, o) AS mask(c1, NULL, q, d, o)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, 'Q', NULL, 'd', 'o') from values ('AbCD123-@$#') as tab(c1) +-- !query analysis +Project [mask(c1#x, Q, null, d, o) AS mask(c1, Q, NULL, d, o)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, 'Q', 'q', NULL, 'o') from values ('AbCD123-@$#') as tab(c1) +-- !query analysis +Project [mask(c1#x, Q, q, null, o) AS mask(c1, Q, q, NULL, o)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, 'Q', 'q', 'd', NULL) from values ('AbCD123-@$#') as tab(c1) +-- !query analysis +Project [mask(c1#x, Q, q, d, null) AS mask(c1, Q, q, d, NULL)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(NULL, 'Q', 'q', 'd', NULL) from values ('AbCD123-@$#') as tab(c1) +-- !query analysis +Project [mask(null, Q, q, d, null) AS mask(NULL, Q, q, d, NULL)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask('abcd-EFGH-8765-4321') +-- !query analysis +Project [mask(abcd-EFGH-8765-4321, X, x, n, null) AS mask(abcd-EFGH-8765-4321, X, x, n, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', 'Q') +-- !query analysis +Project [mask(abcd-EFGH-8765-4321, Q, x, n, null) AS mask(abcd-EFGH-8765-4321, Q, x, n, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', 'Q', 'q') +-- !query analysis +Project [mask(abcd-EFGH-8765-4321, Q, q, n, null) AS mask(abcd-EFGH-8765-4321, Q, q, n, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', 'Q', 'q', 'd') +-- !query analysis +Project [mask(abcd-EFGH-8765-4321, Q, q, d, null) AS mask(abcd-EFGH-8765-4321, Q, q, d, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', 'Q', 'q', 'd', '*') +-- !query analysis +Project [mask(abcd-EFGH-8765-4321, Q, q, d, *) AS mask(abcd-EFGH-8765-4321, Q, q, d, *)#x] ++- OneRowRelation + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', NULL, 'q', 'd', '*') +-- !query analysis +Project [mask(abcd-EFGH-8765-4321, null, q, d, *) AS mask(abcd-EFGH-8765-4321, NULL, q, d, *)#x] ++- OneRowRelation + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', NULL, NULL, 'd', '*') +-- !query analysis +Project [mask(abcd-EFGH-8765-4321, null, null, d, *) AS mask(abcd-EFGH-8765-4321, NULL, NULL, d, *)#x] ++- OneRowRelation + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', NULL, NULL, NULL, '*') +-- !query analysis +Project [mask(abcd-EFGH-8765-4321, null, null, null, *) AS mask(abcd-EFGH-8765-4321, NULL, NULL, NULL, *)#x] ++- OneRowRelation + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', NULL, NULL, NULL, NULL) +-- !query analysis +Project [mask(abcd-EFGH-8765-4321, null, null, null, null) AS mask(abcd-EFGH-8765-4321, NULL, NULL, NULL, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT mask(NULL) +-- !query analysis +Project [mask(null, X, x, n, null) AS mask(NULL, X, x, n, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT mask(NULL, NULL, 'q', 'd', '*') +-- !query analysis +Project [mask(null, null, q, d, *) AS mask(NULL, NULL, q, d, *)#x] ++- OneRowRelation + + +-- !query +SELECT mask(NULL, NULL, NULL, 'd', '*') +-- !query analysis +Project [mask(null, null, null, d, *) AS mask(NULL, NULL, NULL, d, *)#x] ++- OneRowRelation + + +-- !query +SELECT mask(NULL, NULL, NULL, NULL, '*') +-- !query analysis +Project [mask(null, null, null, null, *) AS mask(NULL, NULL, NULL, NULL, *)#x] ++- OneRowRelation + + +-- !query +SELECT mask(NULL, NULL, NULL, NULL, NULL) +-- !query analysis +Project [mask(null, null, null, null, null) AS mask(NULL, NULL, NULL, NULL, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT mask(c1) from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query analysis +Project [mask(c1#x, X, x, n, null) AS mask(c1, X, x, n, NULL)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, 'Q') from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query analysis +Project [mask(c1#x, Q, x, n, null) AS mask(c1, Q, x, n, NULL)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, 'Q', 'q')from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query analysis +Project [mask(c1#x, Q, q, n, null) AS mask(c1, Q, q, n, NULL)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, 'Q', 'q', 'd') from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query analysis +Project [mask(c1#x, Q, q, d, null) AS mask(c1, Q, q, d, NULL)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, 'Q', 'q', 'd', '*') from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query analysis +Project [mask(c1#x, Q, q, d, *) AS mask(c1, Q, q, d, *)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, NULL, 'q', 'd', '*') from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query analysis +Project [mask(c1#x, null, q, d, *) AS mask(c1, NULL, q, d, *)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, NULL, NULL, 'd', '*') from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query analysis +Project [mask(c1#x, null, null, d, *) AS mask(c1, NULL, NULL, d, *)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, NULL, NULL, NULL, '*') from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query analysis +Project [mask(c1#x, null, null, null, *) AS mask(c1, NULL, NULL, NULL, *)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, NULL, NULL, NULL, NULL) from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query analysis +Project [mask(c1#x, null, null, null, null) AS mask(c1, NULL, NULL, NULL, NULL)#x] ++- SubqueryAlias tab + +- LocalRelation [c1#x] + + +-- !query +SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 'a') as t(c1, replaceArg) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"replaceArg\"", + "inputName" : "upperChar", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"mask(c1, replaceArg, x, n, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "mask(c1, replaceArg)" + } ] +} + + +-- !query +SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 'ABC') as t(c1, replaceArg) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"replaceArg\"", + "inputName" : "upperChar", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"mask(c1, replaceArg, x, n, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "mask(c1, replaceArg)" + } ] +} + + +-- !query +SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 123) as t(c1, replaceArg) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"replaceArg\"", + "inputType" : "\"INT\"", + "paramIndex" : "2", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"mask(c1, replaceArg, x, n, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "mask(c1, replaceArg)" + } ] +} + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', 'A', 'w', '') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "digitChar", + "sqlExpr" : "\"mask(abcd-EFGH-8765-4321, A, w, , NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "mask('abcd-EFGH-8765-4321', 'A', 'w', '')" + } ] +} + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', 'A', 'abc') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "lowerChar", + "sqlExpr" : "\"mask(abcd-EFGH-8765-4321, A, abc, n, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "mask('abcd-EFGH-8765-4321', 'A', 'abc')" + } ] +} 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 new file mode 100644 index 0000000000000..e4dd1994b2c9e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/math.sql.out @@ -0,0 +1,433 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT round(25y, 1) +-- !query analysis +Project [round(25, 1) AS round(25, 1)#x] ++- OneRowRelation + + +-- !query +SELECT round(25y, 0) +-- !query analysis +Project [round(25, 0) AS round(25, 0)#x] ++- OneRowRelation + + +-- !query +SELECT round(25y, -1) +-- !query analysis +Project [round(25, -1) AS round(25, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(25y, -2) +-- !query analysis +Project [round(25, -2) AS round(25, -2)#x] ++- OneRowRelation + + +-- !query +SELECT round(25y, -3) +-- !query analysis +Project [round(25, -3) AS round(25, -3)#x] ++- OneRowRelation + + +-- !query +SELECT round(127y, -1) +-- !query analysis +Project [round(127, -1) AS round(127, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(-128y, -1) +-- !query analysis +Project [round(-128, -1) AS round(-128, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(525s, 1) +-- !query analysis +Project [round(525, 1) AS round(525, 1)#x] ++- OneRowRelation + + +-- !query +SELECT round(525s, 0) +-- !query analysis +Project [round(525, 0) AS round(525, 0)#x] ++- OneRowRelation + + +-- !query +SELECT round(525s, -1) +-- !query analysis +Project [round(525, -1) AS round(525, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(525s, -2) +-- !query analysis +Project [round(525, -2) AS round(525, -2)#x] ++- OneRowRelation + + +-- !query +SELECT round(525s, -3) +-- !query analysis +Project [round(525, -3) AS round(525, -3)#x] ++- OneRowRelation + + +-- !query +SELECT round(32767s, -1) +-- !query analysis +Project [round(32767, -1) AS round(32767, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(-32768s, -1) +-- !query analysis +Project [round(-32768, -1) AS round(-32768, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(525, 1) +-- !query analysis +Project [round(525, 1) AS round(525, 1)#x] ++- OneRowRelation + + +-- !query +SELECT round(525, 0) +-- !query analysis +Project [round(525, 0) AS round(525, 0)#x] ++- OneRowRelation + + +-- !query +SELECT round(525, -1) +-- !query analysis +Project [round(525, -1) AS round(525, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(525, -2) +-- !query analysis +Project [round(525, -2) AS round(525, -2)#x] ++- OneRowRelation + + +-- !query +SELECT round(525, -3) +-- !query analysis +Project [round(525, -3) AS round(525, -3)#x] ++- OneRowRelation + + +-- !query +SELECT round(2147483647, -1) +-- !query analysis +Project [round(2147483647, -1) AS round(2147483647, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(-2147483647, -1) +-- !query analysis +Project [round(-2147483647, -1) AS round(-2147483647, -1)#x] ++- OneRowRelation + + +-- !query +SELECT round(525L, 1) +-- !query analysis +Project [round(525, 1) AS round(525, 1)#xL] ++- OneRowRelation + + +-- !query +SELECT round(525L, 0) +-- !query analysis +Project [round(525, 0) AS round(525, 0)#xL] ++- OneRowRelation + + +-- !query +SELECT round(525L, -1) +-- !query analysis +Project [round(525, -1) AS round(525, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT round(525L, -2) +-- !query analysis +Project [round(525, -2) AS round(525, -2)#xL] ++- OneRowRelation + + +-- !query +SELECT round(525L, -3) +-- !query analysis +Project [round(525, -3) AS round(525, -3)#xL] ++- OneRowRelation + + +-- !query +SELECT round(9223372036854775807L, -1) +-- !query analysis +Project [round(9223372036854775807, -1) AS round(9223372036854775807, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT round(-9223372036854775808L, -1) +-- !query analysis +Project [round(-9223372036854775808, -1) AS round(-9223372036854775808, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT bround(25y, 1) +-- !query analysis +Project [bround(25, 1) AS bround(25, 1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(25y, 0) +-- !query analysis +Project [bround(25, 0) AS bround(25, 0)#x] ++- OneRowRelation + + +-- !query +SELECT bround(25y, -1) +-- !query analysis +Project [bround(25, -1) AS bround(25, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(25y, -2) +-- !query analysis +Project [bround(25, -2) AS bround(25, -2)#x] ++- OneRowRelation + + +-- !query +SELECT bround(25y, -3) +-- !query analysis +Project [bround(25, -3) AS bround(25, -3)#x] ++- OneRowRelation + + +-- !query +SELECT bround(127y, -1) +-- !query analysis +Project [bround(127, -1) AS bround(127, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(-128y, -1) +-- !query analysis +Project [bround(-128, -1) AS bround(-128, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525s, 1) +-- !query analysis +Project [bround(525, 1) AS bround(525, 1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525s, 0) +-- !query analysis +Project [bround(525, 0) AS bround(525, 0)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525s, -1) +-- !query analysis +Project [bround(525, -1) AS bround(525, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525s, -2) +-- !query analysis +Project [bround(525, -2) AS bround(525, -2)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525s, -3) +-- !query analysis +Project [bround(525, -3) AS bround(525, -3)#x] ++- OneRowRelation + + +-- !query +SELECT bround(32767s, -1) +-- !query analysis +Project [bround(32767, -1) AS bround(32767, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(-32768s, -1) +-- !query analysis +Project [bround(-32768, -1) AS bround(-32768, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525, 1) +-- !query analysis +Project [bround(525, 1) AS bround(525, 1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525, 0) +-- !query analysis +Project [bround(525, 0) AS bround(525, 0)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525, -1) +-- !query analysis +Project [bround(525, -1) AS bround(525, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525, -2) +-- !query analysis +Project [bround(525, -2) AS bround(525, -2)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525, -3) +-- !query analysis +Project [bround(525, -3) AS bround(525, -3)#x] ++- OneRowRelation + + +-- !query +SELECT bround(2147483647, -1) +-- !query analysis +Project [bround(2147483647, -1) AS bround(2147483647, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(-2147483647, -1) +-- !query analysis +Project [bround(-2147483647, -1) AS bround(-2147483647, -1)#x] ++- OneRowRelation + + +-- !query +SELECT bround(525L, 1) +-- !query analysis +Project [bround(525, 1) AS bround(525, 1)#xL] ++- OneRowRelation + + +-- !query +SELECT bround(525L, 0) +-- !query analysis +Project [bround(525, 0) AS bround(525, 0)#xL] ++- OneRowRelation + + +-- !query +SELECT bround(525L, -1) +-- !query analysis +Project [bround(525, -1) AS bround(525, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT bround(525L, -2) +-- !query analysis +Project [bround(525, -2) AS bround(525, -2)#xL] ++- OneRowRelation + + +-- !query +SELECT bround(525L, -3) +-- !query analysis +Project [bround(525, -3) AS bround(525, -3)#xL] ++- OneRowRelation + + +-- !query +SELECT bround(9223372036854775807L, -1) +-- !query analysis +Project [bround(9223372036854775807, -1) AS bround(9223372036854775807, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT bround(-9223372036854775808L, -1) +-- !query analysis +Project [bround(-9223372036854775808, -1) AS bround(-9223372036854775808, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT conv('100', 2, 10) +-- !query analysis +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, false) 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] ++- OneRowRelation + + +-- !query +SELECT conv('92233720368547758070', 10, 16) +-- !query analysis +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, false) 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] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/misc-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/misc-functions.sql.out new file mode 100644 index 0000000000000..042f2b64f9a7e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/misc-functions.sql.out @@ -0,0 +1,119 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select typeof(null) +-- !query analysis +Project [typeof(null) AS typeof(NULL)#x] ++- OneRowRelation + + +-- !query +select typeof(true) +-- !query analysis +Project [typeof(true) AS typeof(true)#x] ++- OneRowRelation + + +-- !query +select typeof(1Y), typeof(1S), typeof(1), typeof(1L) +-- !query analysis +Project [typeof(1) AS typeof(1)#x, typeof(1) AS typeof(1)#x, typeof(1) AS typeof(1)#x, typeof(1) AS typeof(1)#x] ++- OneRowRelation + + +-- !query +select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2) +-- !query analysis +Project [typeof(cast(1.0 as float)) AS typeof(CAST(1.0 AS FLOAT))#x, typeof(1.0) AS typeof(1.0)#x, typeof(1.2) AS typeof(1.2)#x] ++- OneRowRelation + + +-- !query +select typeof(date '1986-05-23'), typeof(timestamp '1986-05-23'), typeof(interval '23 days') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select typeof(x'ABCD'), typeof('SPARK') +-- !query analysis +Project [typeof(0xABCD) AS typeof(X'ABCD')#x, typeof(SPARK) AS typeof(SPARK)#x] ++- OneRowRelation + + +-- !query +select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark')) +-- !query analysis +Project [typeof(array(1, 2)) AS typeof(array(1, 2))#x, typeof(map(1, 2)) AS typeof(map(1, 2))#x, typeof(named_struct(a, 1, b, spark)) AS typeof(named_struct(a, 1, b, spark))#x] ++- OneRowRelation + + +-- !query +SELECT assert_true(true), assert_true(boolean(1)) +-- !query analysis +Project [assert_true(true, 'true' is not true!) AS assert_true(true, 'true' is not true!)#x, assert_true(cast(1 as boolean), 'cast(1 as boolean)' is not true!) AS assert_true(1, 'cast(1 as boolean)' is not true!)#x] ++- OneRowRelation + + +-- !query +SELECT assert_true(false) +-- !query analysis +Project [assert_true(false, 'false' is not true!) AS assert_true(false, 'false' is not true!)#x] ++- OneRowRelation + + +-- !query +SELECT assert_true(boolean(0)) +-- !query analysis +Project [assert_true(cast(0 as boolean), 'cast(0 as boolean)' is not true!) AS assert_true(0, 'cast(0 as boolean)' is not true!)#x] ++- OneRowRelation + + +-- !query +SELECT assert_true(null) +-- !query analysis +Project [assert_true(null, 'null' is not true!) AS assert_true(NULL, 'null' is not true!)#x] ++- OneRowRelation + + +-- !query +SELECT assert_true(boolean(null)) +-- !query analysis +Project [assert_true(cast(null as boolean), 'cast(null as boolean)' is not true!) AS assert_true(NULL, 'cast(null as boolean)' is not true!)#x] ++- OneRowRelation + + +-- !query +SELECT assert_true(false, 'custom error message') +-- !query analysis +Project [assert_true(false, custom error message) AS assert_true(false, custom error message)#x] ++- OneRowRelation + + +-- !query +CREATE TEMPORARY VIEW tbl_misc AS SELECT * FROM (VALUES (1), (8), (2)) AS T(v) +-- !query analysis +CreateViewCommand `tbl_misc`, SELECT * FROM (VALUES (1), (8), (2)) AS T(v), false, false, LocalTempView, true + +- Project [v#x] + +- SubqueryAlias T + +- Project [col1#x AS v#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT raise_error('error message') +-- !query analysis +Project [raise_error(error message, NullType) AS raise_error(error message)#x] ++- OneRowRelation + + +-- !query +SELECT if(v > 5, raise_error('too big: ' || v), v + 1) FROM tbl_misc +-- !query analysis +Project [if ((v#x > 5)) cast(raise_error(concat(too big: , cast(v#x as string)), NullType) as int) else (v#x + 1) AS (IF((v > 5), raise_error(concat(too big: , v)), (v + 1)))#x] ++- SubqueryAlias tbl_misc + +- View (`tbl_misc`, [v#x]) + +- Project [cast(v#x as int) AS v#x] + +- Project [v#x] + +- SubqueryAlias T + +- Project [col1#x AS v#x] + +- LocalRelation [col1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/natural-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/natural-join.sql.out new file mode 100644 index 0000000000000..8fe2ba77855ba --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/natural-join.sql.out @@ -0,0 +1,680 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query analysis +CreateViewCommand `nt1`, select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1), false, false, LocalTempView, true + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- LocalRelation [k#x, v1#x] + + +-- !query +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query analysis +CreateViewCommand `nt2`, select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2), false, false, LocalTempView, true + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +create temporary view nt3 as select * from values + ("one", 4), + ("two", 5), + ("one", 6) + as nt3(k, v3) +-- !query analysis +CreateViewCommand `nt3`, select * from values + ("one", 4), + ("two", 5), + ("one", 6) + as nt3(k, v3), false, false, LocalTempView, true + +- Project [k#x, v3#x] + +- SubqueryAlias nt3 + +- LocalRelation [k#x, v3#x] + + +-- !query +create temporary view nt4 as select * from values + ("one", 7), + ("two", 8), + ("one", 9) + as nt4(k, v4) +-- !query analysis +CreateViewCommand `nt4`, select * from values + ("one", 7), + ("two", 8), + ("one", 9) + as nt4(k, v4), false, false, LocalTempView, true + +- Project [k#x, v4#x] + +- SubqueryAlias nt4 + +- LocalRelation [k#x, v4#x] + + +-- !query +SELECT * FROM nt1 natural join nt2 +-- !query analysis +Project [k#x, v1#x, v2#x] ++- Project [k#x, v1#x, v2#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 natural join nt2 where k = "one" +-- !query analysis +Project [k#x, v1#x, v2#x] ++- Filter (k#x = one) + +- Project [k#x, v1#x, v2#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 natural left join nt2 order by v1, v2 +-- !query analysis +Sort [v1#x ASC NULLS FIRST, v2#x ASC NULLS FIRST], true ++- Project [k#x, v1#x, v2#x] + +- Project [k#x, v1#x, v2#x] + +- Join LeftOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 natural right join nt2 order by v1, v2 +-- !query analysis +Sort [v1#x ASC NULLS FIRST, v2#x ASC NULLS FIRST], true ++- Project [k#x, v1#x, v2#x] + +- Project [k#x, v1#x, v2#x] + +- Join RightOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT count(*) FROM nt1 natural full outer join nt2 +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Project [coalesce(k#x, k#x) AS k#x, v1#x, v2#x] + +- Join FullOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k FROM nt1 natural join nt2 +-- !query analysis +Project [k#x] ++- Project [k#x, v1#x, v2#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k FROM nt1 natural join nt2 where k = "one" +-- !query analysis +Project [k#x] ++- Filter (k#x = one) + +- Project [k#x, v1#x, v2#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.* FROM nt1 natural join nt2 +-- !query analysis +Project [k#x, v1#x] ++- Project [k#x, v1#x, v2#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt2.* FROM nt1 natural join nt2 +-- !query analysis +Project [k#x, v2#x] ++- Project [k#x, v1#x, v2#x, k#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT sbq.* from (SELECT * FROM nt1 natural join nt2) sbq +-- !query analysis +Project [k#x, v1#x, v2#x] ++- SubqueryAlias sbq + +- Project [k#x, v1#x, v2#x] + +- Project [k#x, v1#x, v2#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT sbq.k from (SELECT * FROM nt1 natural join nt2) sbq +-- !query analysis +Project [k#x] ++- SubqueryAlias sbq + +- Project [k#x, v1#x, v2#x] + +- Project [k#x, v1#x, v2#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.*, nt2.* FROM nt1 natural join nt2 +-- !query analysis +Project [k#x, v1#x, k#x, v2#x] ++- Project [k#x, v1#x, v2#x, k#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT *, nt2.k FROM nt1 natural join nt2 +-- !query analysis +Project [k#x, v1#x, v2#x, k#x] ++- Project [k#x, v1#x, v2#x, k#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.k, nt2.k FROM nt1 natural join nt2 +-- !query analysis +Project [k#x, k#x] ++- Project [k#x, v1#x, v2#x, k#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k FROM (SELECT nt2.k FROM nt1 natural join nt2) +-- !query analysis +Project [k#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [k#x] + +- Project [k#x, v1#x, v2#x, k#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt2.k AS key FROM nt1 natural join nt2 ORDER BY key +-- !query analysis +Sort [key#x ASC NULLS FIRST], true ++- Project [k#x AS key#x] + +- Project [k#x, v1#x, v2#x, k#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.k, nt2.k FROM nt1 natural join nt2 where k = "one" +-- !query analysis +Project [k#x, k#x] ++- Filter (k#x = one) + +- Project [k#x, v1#x, v2#x, k#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM (SELECT * FROM nt1 natural join nt2) +-- !query analysis +Project [k#x, v1#x, v2#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [k#x, v1#x, v2#x] + +- Project [k#x, v1#x, v2#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM (SELECT nt1.*, nt2.* FROM nt1 natural join nt2) +-- !query analysis +Project [k#x, v1#x, k#x, v2#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [k#x, v1#x, k#x, v2#x] + +- Project [k#x, v1#x, v2#x, k#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM (SELECT nt1.v1, nt2.k FROM nt1 natural join nt2) +-- !query analysis +Project [v1#x, k#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [v1#x, k#x] + +- Project [k#x, v1#x, v2#x, k#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt2.k FROM (SELECT * FROM nt1 natural join nt2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`nt2`.`k`", + "proposal" : "`__auto_generated_subquery_name`.`k`, `__auto_generated_subquery_name`.`v1`, `__auto_generated_subquery_name`.`v2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "nt2.k" + } ] +} + + +-- !query +SELECT * FROM nt1 natural join nt2 natural join nt3 +-- !query analysis +Project [k#x, v1#x, v2#x, v3#x] ++- Project [k#x, v1#x, v2#x, v3#x] + +- Join Inner, (k#x = k#x) + :- Project [k#x, v1#x, v2#x] + : +- Join Inner, (k#x = k#x) + : :- SubqueryAlias nt1 + : : +- View (`nt1`, [k#x,v1#x]) + : : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : : +- Project [k#x, v1#x] + : : +- SubqueryAlias nt1 + : : +- LocalRelation [k#x, v1#x] + : +- SubqueryAlias nt2 + : +- View (`nt2`, [k#x,v2#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + : +- Project [k#x, v2#x] + : +- SubqueryAlias nt2 + : +- LocalRelation [k#x, v2#x] + +- SubqueryAlias nt3 + +- View (`nt3`, [k#x,v3#x]) + +- Project [cast(k#x as string) AS k#x, cast(v3#x as int) AS v3#x] + +- Project [k#x, v3#x] + +- SubqueryAlias nt3 + +- LocalRelation [k#x, v3#x] + + +-- !query +SELECT nt1.*, nt2.*, nt3.* FROM nt1 natural join nt2 natural join nt3 +-- !query analysis +Project [k#x, v1#x, k#x, v2#x, k#x, v3#x] ++- Project [k#x, v1#x, v2#x, v3#x, k#x, k#x] + +- Join Inner, (k#x = k#x) + :- Project [k#x, v1#x, v2#x, k#x] + : +- Join Inner, (k#x = k#x) + : :- SubqueryAlias nt1 + : : +- View (`nt1`, [k#x,v1#x]) + : : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : : +- Project [k#x, v1#x] + : : +- SubqueryAlias nt1 + : : +- LocalRelation [k#x, v1#x] + : +- SubqueryAlias nt2 + : +- View (`nt2`, [k#x,v2#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + : +- Project [k#x, v2#x] + : +- SubqueryAlias nt2 + : +- LocalRelation [k#x, v2#x] + +- SubqueryAlias nt3 + +- View (`nt3`, [k#x,v3#x]) + +- Project [cast(k#x as string) AS k#x, cast(v3#x as int) AS v3#x] + +- Project [k#x, v3#x] + +- SubqueryAlias nt3 + +- LocalRelation [k#x, v3#x] + + +-- !query +SELECT nt1.*, nt2.*, nt3.* FROM nt1 natural join nt2 join nt3 on nt2.k = nt3.k +-- !query analysis +Project [k#x, v1#x, k#x, v2#x, k#x, v3#x] ++- Join Inner, (k#x = k#x) + :- Project [k#x, v1#x, v2#x, k#x] + : +- Join Inner, (k#x = k#x) + : :- SubqueryAlias nt1 + : : +- View (`nt1`, [k#x,v1#x]) + : : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : : +- Project [k#x, v1#x] + : : +- SubqueryAlias nt1 + : : +- LocalRelation [k#x, v1#x] + : +- SubqueryAlias nt2 + : +- View (`nt2`, [k#x,v2#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + : +- Project [k#x, v2#x] + : +- SubqueryAlias nt2 + : +- LocalRelation [k#x, v2#x] + +- SubqueryAlias nt3 + +- View (`nt3`, [k#x,v3#x]) + +- Project [cast(k#x as string) AS k#x, cast(v3#x as int) AS v3#x] + +- Project [k#x, v3#x] + +- SubqueryAlias nt3 + +- LocalRelation [k#x, v3#x] + + +-- !query +SELECT * FROM nt1 natural join nt2 join nt3 on nt1.k = nt3.k +-- !query analysis +Project [k#x, v1#x, v2#x, k#x, v3#x] ++- Join Inner, (k#x = k#x) + :- Project [k#x, v1#x, v2#x] + : +- Join Inner, (k#x = k#x) + : :- SubqueryAlias nt1 + : : +- View (`nt1`, [k#x,v1#x]) + : : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : : +- Project [k#x, v1#x] + : : +- SubqueryAlias nt1 + : : +- LocalRelation [k#x, v1#x] + : +- SubqueryAlias nt2 + : +- View (`nt2`, [k#x,v2#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + : +- Project [k#x, v2#x] + : +- SubqueryAlias nt2 + : +- LocalRelation [k#x, v2#x] + +- SubqueryAlias nt3 + +- View (`nt3`, [k#x,v3#x]) + +- Project [cast(k#x as string) AS k#x, cast(v3#x as int) AS v3#x] + +- Project [k#x, v3#x] + +- SubqueryAlias nt3 + +- LocalRelation [k#x, v3#x] + + +-- !query +SELECT * FROM nt1 natural join nt2 join nt3 on nt2.k = nt3.k +-- !query analysis +Project [k#x, v1#x, v2#x, k#x, v3#x] ++- Project [k#x, v1#x, v2#x, k#x, v3#x] + +- Join Inner, (k#x = k#x) + :- Project [k#x, v1#x, v2#x, k#x] + : +- Join Inner, (k#x = k#x) + : :- SubqueryAlias nt1 + : : +- View (`nt1`, [k#x,v1#x]) + : : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : : +- Project [k#x, v1#x] + : : +- SubqueryAlias nt1 + : : +- LocalRelation [k#x, v1#x] + : +- SubqueryAlias nt2 + : +- View (`nt2`, [k#x,v2#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + : +- Project [k#x, v2#x] + : +- SubqueryAlias nt2 + : +- LocalRelation [k#x, v2#x] + +- SubqueryAlias nt3 + +- View (`nt3`, [k#x,v3#x]) + +- Project [cast(k#x as string) AS k#x, cast(v3#x as int) AS v3#x] + +- Project [k#x, v3#x] + +- SubqueryAlias nt3 + +- LocalRelation [k#x, v3#x] + + +-- !query +SELECT nt1.*, nt2.*, nt3.*, nt4.* FROM nt1 natural join nt2 natural join nt3 natural join nt4 +-- !query analysis +Project [k#x, v1#x, k#x, v2#x, k#x, v3#x, k#x, v4#x] ++- Project [k#x, v1#x, v2#x, v3#x, v4#x, k#x, k#x, k#x] + +- Join Inner, (k#x = k#x) + :- Project [k#x, v1#x, v2#x, v3#x, k#x, k#x] + : +- Join Inner, (k#x = k#x) + : :- Project [k#x, v1#x, v2#x, k#x] + : : +- Join Inner, (k#x = k#x) + : : :- SubqueryAlias nt1 + : : : +- View (`nt1`, [k#x,v1#x]) + : : : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : : : +- Project [k#x, v1#x] + : : : +- SubqueryAlias nt1 + : : : +- LocalRelation [k#x, v1#x] + : : +- SubqueryAlias nt2 + : : +- View (`nt2`, [k#x,v2#x]) + : : +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + : : +- Project [k#x, v2#x] + : : +- SubqueryAlias nt2 + : : +- LocalRelation [k#x, v2#x] + : +- SubqueryAlias nt3 + : +- View (`nt3`, [k#x,v3#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v3#x as int) AS v3#x] + : +- Project [k#x, v3#x] + : +- SubqueryAlias nt3 + : +- LocalRelation [k#x, v3#x] + +- SubqueryAlias nt4 + +- View (`nt4`, [k#x,v4#x]) + +- Project [cast(k#x as string) AS k#x, cast(v4#x as int) AS v4#x] + +- Project [k#x, v4#x] + +- SubqueryAlias nt4 + +- LocalRelation [k#x, v4#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out new file mode 100644 index 0000000000000..305a59f01e443 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out @@ -0,0 +1,72 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SET spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ReplaceCTERefWithRepartition +-- !query analysis +SetCommand (spark.sql.optimizer.excludedRules,Some(org.apache.spark.sql.catalyst.optimizer.ReplaceCTERefWithRepartition)) + + +-- !query +SELECT + (SELECT min(id) FROM range(10)), + (SELECT sum(id) FROM range(10)), + (SELECT count(distinct id) FROM range(10)) +-- !query analysis +Project [scalar-subquery#x [] AS scalarsubquery()#xL, scalar-subquery#x [] AS scalarsubquery()#xL, scalar-subquery#x [] AS scalarsubquery()#xL] +: :- Aggregate [min(id#xL) AS min(id)#xL] +: : +- Range (0, 10, step=1, splits=None) +: :- Aggregate [sum(id#xL) AS sum(id)#xL] +: : +- Range (0, 10, step=1, splits=None) +: +- Aggregate [count(distinct id#xL) AS count(DISTINCT id)#xL] +: +- Range (0, 10, step=1, splits=None) ++- OneRowRelation + + +-- !query +SET spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.Optimizer$OptimizeSubqueries +-- !query analysis +SetCommand (spark.sql.optimizer.excludedRules,Some(org.apache.spark.sql.catalyst.optimizer.Optimizer$OptimizeSubqueries)) + + +-- !query +WITH tmp AS ( + SELECT id FROM range(2) + INTERSECT + SELECT id FROM range(4) +) +SELECT id FROM range(3) WHERE id > (SELECT max(id) FROM tmp) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias tmp +: +- Intersect false +: :- Project [id#xL] +: : +- Range (0, 2, step=1, splits=None) +: +- Project [id#xL] +: +- Range (0, 4, step=1, splits=None) ++- Project [id#xL] + +- Filter (id#xL > scalar-subquery#x []) + : +- Aggregate [max(id#xL) AS max(id)#xL] + : +- SubqueryAlias tmp + : +- CTERelationRef xxxx, true, [id#xL] + +- Range (0, 3, step=1, splits=None) + + +-- !query +SET spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.RewriteLateralSubquery +-- !query analysis +SetCommand (spark.sql.optimizer.excludedRules,Some(org.apache.spark.sql.catalyst.optimizer.RewriteLateralSubquery)) + + +-- !query +SELECT * FROM testData, LATERAL (SELECT * FROM testData) LIMIT 1 +-- !query analysis +GlobalLimit 1 ++- LocalLimit 1 + +- Project [key#x, value#x, key#x, value#x] + +- LateralJoin lateral-subquery#x [], Inner + : +- SubqueryAlias __auto_generated_subquery_name + : +- Project [key#x, value#x] + : +- SubqueryAlias spark_catalog.default.testdata + : +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out new file mode 100644 index 0000000000000..02807140ed361 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out @@ -0,0 +1,239 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create table t1(a int, b int, c int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false + + +-- !query +insert into t1 values(1,0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t1 values(2,0,1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t1 values(3,1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t1 values(4,1,1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t1 values(5,null,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t1 values(6,null,1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t1 values(7,null,null) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +select a, b+c from t1 +-- !query analysis +Project [a#x, (b#x + c#x) AS (b + c)#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select a+10, b*0 from t1 +-- !query analysis +Project [(a#x + 10) AS (a + 10)#x, (b#x * 0) AS (b * 0)#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select distinct b from t1 +-- !query analysis +Distinct ++- Project [b#x] + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select b from t1 union select b from t1 +-- !query analysis +Distinct ++- Union false, false + :- Project [b#x] + : +- SubqueryAlias spark_catalog.default.t1 + : +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + +- Project [b#x] + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select a+20, case b when c then 1 else 0 end from t1 +-- !query analysis +Project [(a#x + 20) AS (a + 20)#x, CASE WHEN (b#x = c#x) THEN 1 ELSE 0 END AS CASE WHEN (b = c) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select a+30, case c when b then 1 else 0 end from t1 +-- !query analysis +Project [(a#x + 30) AS (a + 30)#x, CASE WHEN (c#x = b#x) THEN 1 ELSE 0 END AS CASE WHEN (c = b) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select a+40, case when b<>0 then 1 else 0 end from t1 +-- !query analysis +Project [(a#x + 40) AS (a + 40)#x, CASE WHEN NOT (b#x = 0) THEN 1 ELSE 0 END AS CASE WHEN (NOT (b = 0)) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select a+50, case when not b<>0 then 1 else 0 end from t1 +-- !query analysis +Project [(a#x + 50) AS (a + 50)#x, CASE WHEN NOT NOT (b#x = 0) THEN 1 ELSE 0 END AS CASE WHEN (NOT (NOT (b = 0))) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select a+60, case when b<>0 and c<>0 then 1 else 0 end from t1 +-- !query analysis +Project [(a#x + 60) AS (a + 60)#x, CASE WHEN (NOT (b#x = 0) AND NOT (c#x = 0)) THEN 1 ELSE 0 END AS CASE WHEN ((NOT (b = 0)) AND (NOT (c = 0))) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select a+70, case when not (b<>0 and c<>0) then 1 else 0 end from t1 +-- !query analysis +Project [(a#x + 70) AS (a + 70)#x, CASE WHEN NOT (NOT (b#x = 0) AND NOT (c#x = 0)) THEN 1 ELSE 0 END AS CASE WHEN (NOT ((NOT (b = 0)) AND (NOT (c = 0)))) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select a+80, case when b<>0 or c<>0 then 1 else 0 end from t1 +-- !query analysis +Project [(a#x + 80) AS (a + 80)#x, CASE WHEN (NOT (b#x = 0) OR NOT (c#x = 0)) THEN 1 ELSE 0 END AS CASE WHEN ((NOT (b = 0)) OR (NOT (c = 0))) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select a+90, case when not (b<>0 or c<>0) then 1 else 0 end from t1 +-- !query analysis +Project [(a#x + 90) AS (a + 90)#x, CASE WHEN NOT (NOT (b#x = 0) OR NOT (c#x = 0)) THEN 1 ELSE 0 END AS CASE WHEN (NOT ((NOT (b = 0)) OR (NOT (c = 0)))) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select count(*), count(b), sum(b), avg(b), min(b), max(b) from t1 +-- !query analysis +Aggregate [count(1) AS count(1)#xL, count(b#x) AS count(b)#xL, sum(b#x) AS sum(b)#xL, avg(b#x) AS avg(b)#x, min(b#x) AS min(b)#x, max(b#x) AS max(b)#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select a+100 from t1 where b<10 +-- !query analysis +Project [(a#x + 100) AS (a + 100)#x] ++- Filter (b#x < 10) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select a+110 from t1 where not b>10 +-- !query analysis +Project [(a#x + 110) AS (a + 110)#x] ++- Filter NOT (b#x > 10) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select a+120 from t1 where b<10 OR c=1 +-- !query analysis +Project [(a#x + 120) AS (a + 120)#x] ++- Filter ((b#x < 10) OR (c#x = 1)) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select a+130 from t1 where b<10 AND c=1 +-- !query analysis +Project [(a#x + 130) AS (a + 130)#x] ++- Filter ((b#x < 10) AND (c#x = 1)) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select a+140 from t1 where not (b<10 AND c=1) +-- !query analysis +Project [(a#x + 140) AS (a + 140)#x] ++- Filter NOT ((b#x < 10) AND (c#x = 1)) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select a+150 from t1 where not (c=1 AND b<10) +-- !query analysis +Project [(a#x + 150) AS (a + 150)#x] ++- Filter NOT ((c#x = 1) AND (b#x < 10)) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select b, c, equal_null(b, c), equal_null(c, b) from t1 +-- !query analysis +Project [b#x, c#x, equal_null(b#x, c#x) AS equal_null(b, c)#x, equal_null(c#x, b#x) AS equal_null(c, b)#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +drop table t1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t1`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/null-propagation.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/null-propagation.sql.out new file mode 100644 index 0000000000000..39f4d1d12ed04 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/null-propagation.sql.out @@ -0,0 +1,33 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT COUNT(NULL) FROM VALUES 1, 2, 3 +-- !query analysis +Aggregate [count(null) AS count(NULL)#xL] ++- LocalRelation [col1#x] + + +-- !query +SELECT COUNT(1 + NULL) FROM VALUES 1, 2, 3 +-- !query analysis +Aggregate [count((1 + cast(null as int))) AS count((1 + NULL))#xL] ++- LocalRelation [col1#x] + + +-- !query +SELECT COUNT(NULL) OVER () FROM VALUES 1, 2, 3 +-- !query analysis +Project [count(NULL) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] ++- Project [count(NULL) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, count(NULL) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] + +- Window [count(null) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS count(NULL) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] + +- Project + +- LocalRelation [col1#x] + + +-- !query +SELECT COUNT(1 + NULL) OVER () FROM VALUES 1, 2, 3 +-- !query analysis +Project [count((1 + NULL)) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] ++- Project [count((1 + NULL)) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, count((1 + NULL)) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] + +- Window [count((1 + cast(null as int))) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS count((1 + NULL)) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] + +- Project + +- LocalRelation [col1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/operators.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/operators.sql.out new file mode 100644 index 0000000000000..78e56fb09093c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/operators.sql.out @@ -0,0 +1,549 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select -100 +-- !query analysis +Project [-100 AS -100#x] ++- OneRowRelation + + +-- !query +select +230 +-- !query analysis +Project [positive(230) AS (+ 230)#x] ++- OneRowRelation + + +-- !query +select -5.2 +-- !query analysis +Project [-5.2 AS -5.2#x] ++- OneRowRelation + + +-- !query +select +6.8e0 +-- !query analysis +Project [positive(6.8) AS (+ 6.8)#x] ++- OneRowRelation + + +-- !query +select -key, +key from testdata where key = 2 +-- !query analysis +Project [-key#x AS (- key)#x, positive(key#x) AS (+ key)#x] ++- Filter (key#x = 2) + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +select -(key + 1), - key + 1, +(key + 5) from testdata where key = 1 +-- !query analysis +Project [-(key#x + 1) AS (- (key + 1))#x, (-key#x + 1) AS ((- key) + 1)#x, positive((key#x + 5)) AS (+ (key + 5))#x] ++- Filter (key#x = 1) + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +select -max(key), +max(key) from testdata +-- !query analysis +Aggregate [-max(key#x) AS (- max(key))#x, positive(max(key#x)) AS (+ max(key))#x] ++- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +select - (-10) +-- !query analysis +Project [--10 AS (- -10)#x] ++- OneRowRelation + + +-- !query +select + (-key) from testdata where key = 32 +-- !query analysis +Project [positive(-key#x) AS (+ (- key))#x] ++- Filter (key#x = 32) + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +select - (+max(key)) from testdata +-- !query analysis +Aggregate [-positive(max(key#x)) AS (- (+ max(key)))#x] ++- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +select - - 3 +-- !query analysis +Project [--3 AS (- -3)#x] ++- OneRowRelation + + +-- !query +select - + 20 +-- !query analysis +Project [-positive(20) AS (- (+ 20))#x] ++- OneRowRelation + + +-- !query +select + + 100 +-- !query analysis +Project [positive(positive(100)) AS (+ (+ 100))#x] ++- OneRowRelation + + +-- !query +select - - max(key) from testdata +-- !query analysis +Aggregate [--max(key#x) AS (- (- max(key)))#x] ++- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +select + - key from testdata where key = 33 +-- !query analysis +Project [positive(-key#x) AS (+ (- key))#x] ++- Filter (key#x = 33) + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +select 5 / 2 +-- !query analysis +Project [(cast(5 as double) / cast(2 as double)) AS (5 / 2)#x] ++- OneRowRelation + + +-- !query +select 5 / 0 +-- !query analysis +Project [(cast(5 as double) / cast(0 as double)) AS (5 / 0)#x] ++- OneRowRelation + + +-- !query +select 5 / null +-- !query analysis +Project [(cast(5 as double) / cast(null as double)) AS (5 / NULL)#x] ++- OneRowRelation + + +-- !query +select null / 5 +-- !query analysis +Project [(cast(null as double) / cast(5 as double)) AS (NULL / 5)#x] ++- OneRowRelation + + +-- !query +select 5 div 2 +-- !query analysis +Project [(cast(5 as bigint) div cast(2 as bigint)) AS (5 div 2)#xL] ++- OneRowRelation + + +-- !query +select 5 div 0 +-- !query analysis +Project [(cast(5 as bigint) div cast(0 as bigint)) AS (5 div 0)#xL] ++- OneRowRelation + + +-- !query +select 5 div null +-- !query analysis +Project [(cast(5 as bigint) div cast(null as bigint)) AS (5 div NULL)#xL] ++- OneRowRelation + + +-- !query +select null div 5 +-- !query analysis +Project [(cast(null as bigint) div cast(5 as bigint)) AS (NULL div 5)#xL] ++- OneRowRelation + + +-- !query +select cast(51 as decimal(10, 0)) div cast(2 as decimal(2, 0)) +-- !query analysis +Project [(cast(51 as decimal(10,0)) div cast(2 as decimal(2,0))) AS (CAST(51 AS DECIMAL(10,0)) div CAST(2 AS DECIMAL(2,0)))#xL] ++- OneRowRelation + + +-- !query +select cast(5 as decimal(1, 0)) div cast(0 as decimal(2, 0)) +-- !query analysis +Project [(cast(5 as decimal(1,0)) div cast(0 as decimal(2,0))) AS (CAST(5 AS DECIMAL(1,0)) div CAST(0 AS DECIMAL(2,0)))#xL] ++- OneRowRelation + + +-- !query +select cast(5 as decimal(1, 0)) div cast(null as decimal(2, 0)) +-- !query analysis +Project [(cast(5 as decimal(1,0)) div cast(null as decimal(2,0))) AS (CAST(5 AS DECIMAL(1,0)) div CAST(NULL AS DECIMAL(2,0)))#xL] ++- OneRowRelation + + +-- !query +select cast(null as decimal(1, 0)) div cast(5 as decimal(2, 0)) +-- !query analysis +Project [(cast(null as decimal(1,0)) div cast(5 as decimal(2,0))) AS (CAST(NULL AS DECIMAL(1,0)) div CAST(5 AS DECIMAL(2,0)))#xL] ++- OneRowRelation + + +-- !query +select 1 + 2 +-- !query analysis +Project [(1 + 2) AS (1 + 2)#x] ++- OneRowRelation + + +-- !query +select 1 - 2 +-- !query analysis +Project [(1 - 2) AS (1 - 2)#x] ++- OneRowRelation + + +-- !query +select 2 * 5 +-- !query analysis +Project [(2 * 5) AS (2 * 5)#x] ++- OneRowRelation + + +-- !query +select 5 % 3 +-- !query analysis +Project [(5 % 3) AS (5 % 3)#x] ++- OneRowRelation + + +-- !query +select pmod(-7, 3) +-- !query analysis +Project [pmod(-7, 3) AS pmod(-7, 3)#x] ++- OneRowRelation + + +-- !query +select sec(1) +-- !query analysis +Project [SEC(cast(1 as double)) AS SEC(1)#x] ++- OneRowRelation + + +-- !query +select sec(null) +-- !query analysis +Project [SEC(cast(null as double)) AS SEC(NULL)#x] ++- OneRowRelation + + +-- !query +select sec(0) +-- !query analysis +Project [SEC(cast(0 as double)) AS SEC(0)#x] ++- OneRowRelation + + +-- !query +select sec(-1) +-- !query analysis +Project [SEC(cast(-1 as double)) AS SEC(-1)#x] ++- OneRowRelation + + +-- !query +select csc(1) +-- !query analysis +Project [CSC(cast(1 as double)) AS CSC(1)#x] ++- OneRowRelation + + +-- !query +select csc(null) +-- !query analysis +Project [CSC(cast(null as double)) AS CSC(NULL)#x] ++- OneRowRelation + + +-- !query +select csc(0) +-- !query analysis +Project [CSC(cast(0 as double)) AS CSC(0)#x] ++- OneRowRelation + + +-- !query +select csc(-1) +-- !query analysis +Project [CSC(cast(-1 as double)) AS CSC(-1)#x] ++- OneRowRelation + + +-- !query +select cot(1) +-- !query analysis +Project [COT(cast(1 as double)) AS COT(1)#x] ++- OneRowRelation + + +-- !query +select cot(null) +-- !query analysis +Project [COT(cast(null as double)) AS COT(NULL)#x] ++- OneRowRelation + + +-- !query +select cot(0) +-- !query analysis +Project [COT(cast(0 as double)) AS COT(0)#x] ++- OneRowRelation + + +-- !query +select cot(-1) +-- !query analysis +Project [COT(cast(-1 as double)) AS COT(-1)#x] ++- OneRowRelation + + +-- !query +select ceiling(0) +-- !query analysis +Project [ceiling(cast(0 as double)) AS ceiling(0)#xL] ++- OneRowRelation + + +-- !query +select ceiling(1) +-- !query analysis +Project [ceiling(cast(1 as double)) AS ceiling(1)#xL] ++- OneRowRelation + + +-- !query +select ceil(1234567890123456) +-- !query analysis +Project [CEIL(1234567890123456) AS CEIL(1234567890123456)#xL] ++- OneRowRelation + + +-- !query +select ceiling(1234567890123456) +-- !query analysis +Project [ceiling(1234567890123456) AS ceiling(1234567890123456)#xL] ++- OneRowRelation + + +-- !query +select ceil(0.01) +-- !query analysis +Project [CEIL(0.01) AS CEIL(0.01)#x] ++- OneRowRelation + + +-- !query +select ceiling(-0.10) +-- !query analysis +Project [ceiling(-0.10) AS ceiling(-0.10)#x] ++- OneRowRelation + + +-- !query +select floor(0) +-- !query analysis +Project [FLOOR(cast(0 as double)) AS FLOOR(0)#xL] ++- OneRowRelation + + +-- !query +select floor(1) +-- !query analysis +Project [FLOOR(cast(1 as double)) AS FLOOR(1)#xL] ++- OneRowRelation + + +-- !query +select floor(1234567890123456) +-- !query analysis +Project [FLOOR(1234567890123456) AS FLOOR(1234567890123456)#xL] ++- OneRowRelation + + +-- !query +select floor(0.01) +-- !query analysis +Project [FLOOR(0.01) AS FLOOR(0.01)#x] ++- OneRowRelation + + +-- !query +select floor(-0.10) +-- !query analysis +Project [FLOOR(-0.10) AS FLOOR(-0.10)#x] ++- OneRowRelation + + +-- !query +select 1 > 0.00001 +-- !query analysis +Project [(cast(1 as bigint) > 0) AS (1 > 0)#x] ++- OneRowRelation + + +-- !query +select mod(7, 2), mod(7, 0), mod(0, 2), mod(7, null), mod(null, 2), mod(null, null) +-- !query analysis +Project [mod(7, 2) AS mod(7, 2)#x, mod(7, 0) AS mod(7, 0)#x, mod(0, 2) AS mod(0, 2)#x, mod(7, cast(null as int)) AS mod(7, NULL)#x, mod(cast(null as int), 2) AS mod(NULL, 2)#x, mod(null, null) AS mod(NULL, NULL)#x] ++- OneRowRelation + + +-- !query +select BIT_LENGTH('abc') +-- !query analysis +Project [bit_length(abc) AS bit_length(abc)#x] ++- OneRowRelation + + +-- !query +select CHAR_LENGTH('abc') +-- !query analysis +Project [char_length(abc) AS char_length(abc)#x] ++- OneRowRelation + + +-- !query +select CHARACTER_LENGTH('abc') +-- !query analysis +Project [character_length(abc) AS character_length(abc)#x] ++- OneRowRelation + + +-- !query +select OCTET_LENGTH('abc') +-- !query analysis +Project [octet_length(abc) AS octet_length(abc)#x] ++- OneRowRelation + + +-- !query +select abs(-3.13), abs('-2.19') +-- !query analysis +Project [abs(-3.13) AS abs(-3.13)#x, abs(cast(-2.19 as double)) AS abs(-2.19)#x] ++- OneRowRelation + + +-- !query +select positive('-1.11'), positive(-1.11), negative('-1.11'), negative(-1.11) +-- !query analysis +Project [positive(cast(-1.11 as double)) AS (+ -1.11)#x, positive(-1.11) AS (+ -1.11)#x, -cast(-1.11 as double) AS negative(-1.11)#x, --1.11 AS negative(-1.11)#x] ++- OneRowRelation + + +-- !query +select pmod(-7, 2), pmod(0, 2), pmod(7, 0), pmod(7, null), pmod(null, 2), pmod(null, null) +-- !query analysis +Project [pmod(-7, 2) AS pmod(-7, 2)#x, pmod(0, 2) AS pmod(0, 2)#x, pmod(7, 0) AS pmod(7, 0)#x, pmod(7, cast(null as int)) AS pmod(7, NULL)#x, pmod(cast(null as int), 2) AS pmod(NULL, 2)#x, pmod(null, null) AS pmod(NULL, NULL)#x] ++- OneRowRelation + + +-- !query +select pmod(cast(3.13 as decimal), cast(0 as decimal)), pmod(cast(2 as smallint), cast(0 as smallint)) +-- !query analysis +Project [pmod(cast(3.13 as decimal(10,0)), cast(0 as decimal(10,0))) AS pmod(CAST(3.13 AS DECIMAL(10,0)), CAST(0 AS DECIMAL(10,0)))#x, pmod(cast(2 as smallint), cast(0 as smallint)) AS pmod(CAST(2 AS SMALLINT), CAST(0 AS SMALLINT))#x] ++- OneRowRelation + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 5) +-- !query analysis +Project [width_bucket(cast(5.35 as double), cast(0.024 as double), cast(10.06 as double), cast(5 as bigint)) AS width_bucket(5.35, 0.024, 10.06, 5)#xL] ++- OneRowRelation + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 3 + 2) +-- !query analysis +Project [width_bucket(cast(5.35 as double), cast(0.024 as double), cast(10.06 as double), cast((3 + 2) as bigint)) AS width_bucket(5.35, 0.024, 10.06, (3 + 2))#xL] ++- OneRowRelation + + +-- !query +select width_bucket('5.35', '0.024', '10.06', '5') +-- !query analysis +Project [width_bucket(cast(5.35 as double), cast(0.024 as double), cast(10.06 as double), cast(5 as bigint)) AS width_bucket(5.35, 0.024, 10.06, 5)#xL] ++- OneRowRelation + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 2.5) +-- !query analysis +Project [width_bucket(cast(5.35 as double), cast(0.024 as double), cast(10.06 as double), cast(2.5 as bigint)) AS width_bucket(5.35, 0.024, 10.06, 2.5)#xL] ++- OneRowRelation + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 0.5) +-- !query analysis +Project [width_bucket(cast(5.35 as double), cast(0.024 as double), cast(10.06 as double), cast(0.5 as bigint)) AS width_bucket(5.35, 0.024, 10.06, 0.5)#xL] ++- OneRowRelation + + +-- !query +select width_bucket(null, 0.024, 10.06, 5) +-- !query analysis +Project [width_bucket(cast(null as double), cast(0.024 as double), cast(10.06 as double), cast(5 as bigint)) AS width_bucket(NULL, 0.024, 10.06, 5)#xL] ++- OneRowRelation + + +-- !query +select width_bucket(5.35, null, 10.06, 5) +-- !query analysis +Project [width_bucket(cast(5.35 as double), cast(null as double), cast(10.06 as double), cast(5 as bigint)) AS width_bucket(5.35, NULL, 10.06, 5)#xL] ++- OneRowRelation + + +-- !query +select width_bucket(5.35, 0.024, null, -5) +-- !query analysis +Project [width_bucket(cast(5.35 as double), cast(0.024 as double), cast(null as double), cast(-5 as bigint)) AS width_bucket(5.35, 0.024, NULL, -5)#xL] ++- OneRowRelation + + +-- !query +select width_bucket(5.35, 0.024, 10.06, null) +-- !query analysis +Project [width_bucket(cast(5.35 as double), cast(0.024 as double), cast(10.06 as double), cast(null as bigint)) AS width_bucket(5.35, 0.024, 10.06, NULL)#xL] ++- OneRowRelation + + +-- !query +select width_bucket(5.35, 0.024, 10.06, -5) +-- !query analysis +Project [width_bucket(cast(5.35 as double), cast(0.024 as double), cast(10.06 as double), cast(-5 as bigint)) AS width_bucket(5.35, 0.024, 10.06, -5)#xL] ++- OneRowRelation + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 9223372036854775807L) +-- !query analysis +Project [width_bucket(cast(5.35 as double), cast(0.024 as double), cast(10.06 as double), 9223372036854775807) AS width_bucket(5.35, 0.024, 10.06, 9223372036854775807)#xL] ++- OneRowRelation + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 9223372036854775807L - 1) +-- !query analysis +Project [width_bucket(cast(5.35 as double), cast(0.024 as double), cast(10.06 as double), (9223372036854775807 - cast(1 as bigint))) AS width_bucket(5.35, 0.024, 10.06, (9223372036854775807 - 1))#xL] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-all.sql.out new file mode 100644 index 0000000000000..dd4f61e3c9962 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-all.sql.out @@ -0,0 +1,250 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view data as select * from values + (0, 1), + (0, 2), + (1, 3), + (1, NULL) + as data(g, i) +-- !query analysis +CreateViewCommand `data`, select * from values + (0, 1), + (0, 2), + (1, 3), + (1, NULL) + as data(g, i), false, false, LocalTempView, true + +- Project [g#x, i#x] + +- SubqueryAlias data + +- LocalRelation [g#x, i#x] + + +-- !query +select g from data order by all +-- !query analysis +Sort [g#x ASC NULLS FIRST], true ++- Project [g#x] + +- SubqueryAlias data + +- View (`data`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- LocalRelation [g#x, i#x] + + +-- !query +select * from data order by all +-- !query analysis +Sort [g#x ASC NULLS FIRST, i#x ASC NULLS FIRST], true ++- Project [g#x, i#x] + +- SubqueryAlias data + +- View (`data`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- LocalRelation [g#x, i#x] + + +-- !query +select * from data order by aLl +-- !query analysis +Sort [g#x ASC NULLS FIRST, i#x ASC NULLS FIRST], true ++- Project [g#x, i#x] + +- SubqueryAlias data + +- View (`data`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- LocalRelation [g#x, i#x] + + +-- !query +select * from data order by all asc +-- !query analysis +Sort [g#x ASC NULLS FIRST, i#x ASC NULLS FIRST], true ++- Project [g#x, i#x] + +- SubqueryAlias data + +- View (`data`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- LocalRelation [g#x, i#x] + + +-- !query +select * from data order by all desc +-- !query analysis +Sort [g#x DESC NULLS LAST, i#x DESC NULLS LAST], true ++- Project [g#x, i#x] + +- SubqueryAlias data + +- View (`data`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- LocalRelation [g#x, i#x] + + +-- !query +select * from data order by all nulls first +-- !query analysis +Sort [g#x ASC NULLS FIRST, i#x ASC NULLS FIRST], true ++- Project [g#x, i#x] + +- SubqueryAlias data + +- View (`data`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- LocalRelation [g#x, i#x] + + +-- !query +select * from data order by all nulls last +-- !query analysis +Sort [g#x ASC NULLS LAST, i#x ASC NULLS LAST], true ++- Project [g#x, i#x] + +- SubqueryAlias data + +- View (`data`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- LocalRelation [g#x, i#x] + + +-- !query +select * from data order by all asc nulls first +-- !query analysis +Sort [g#x ASC NULLS FIRST, i#x ASC NULLS FIRST], true ++- Project [g#x, i#x] + +- SubqueryAlias data + +- View (`data`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- LocalRelation [g#x, i#x] + + +-- !query +select * from data order by all desc nulls first +-- !query analysis +Sort [g#x DESC NULLS FIRST, i#x DESC NULLS FIRST], true ++- Project [g#x, i#x] + +- SubqueryAlias data + +- View (`data`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- LocalRelation [g#x, i#x] + + +-- !query +select * from data order by all asc nulls last +-- !query analysis +Sort [g#x ASC NULLS LAST, i#x ASC NULLS LAST], true ++- Project [g#x, i#x] + +- SubqueryAlias data + +- View (`data`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- LocalRelation [g#x, i#x] + + +-- !query +select * from data order by all desc nulls last +-- !query analysis +Sort [g#x DESC NULLS LAST, i#x DESC NULLS LAST], true ++- Project [g#x, i#x] + +- SubqueryAlias data + +- View (`data`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- LocalRelation [g#x, i#x] + + +-- !query +select * from data union all select * from data order by all +-- !query analysis +Sort [g#x ASC NULLS FIRST, i#x ASC NULLS FIRST], true ++- Union false, false + :- Project [g#x, i#x] + : +- SubqueryAlias data + : +- View (`data`, [g#x,i#x]) + : +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + : +- Project [g#x, i#x] + : +- SubqueryAlias data + : +- LocalRelation [g#x, i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- View (`data`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- LocalRelation [g#x, i#x] + + +-- !query +select * from data union select * from data order by all +-- !query analysis +Sort [g#x ASC NULLS FIRST, i#x ASC NULLS FIRST], true ++- Distinct + +- Union false, false + :- Project [g#x, i#x] + : +- SubqueryAlias data + : +- View (`data`, [g#x,i#x]) + : +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + : +- Project [g#x, i#x] + : +- SubqueryAlias data + : +- LocalRelation [g#x, i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- View (`data`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- LocalRelation [g#x, i#x] + + +-- !query +select * from data order by all limit 2 +-- !query analysis +GlobalLimit 2 ++- LocalLimit 2 + +- Sort [g#x ASC NULLS FIRST, i#x ASC NULLS FIRST], true + +- Project [g#x, i#x] + +- SubqueryAlias data + +- View (`data`, [g#x,i#x]) + +- Project [cast(g#x as int) AS g#x, cast(i#x as int) AS i#x] + +- Project [g#x, i#x] + +- SubqueryAlias data + +- LocalRelation [g#x, i#x] + + +-- !query +select * from values("z", 1), ("y", 2), ("x", 3) AS T(col1, all) order by all +-- !query analysis +Sort [all#x ASC NULLS FIRST], true ++- Project [col1#x, all#x] + +- SubqueryAlias T + +- LocalRelation [col1#x, all#x] + + +-- !query +select name, dept, rank() over (partition by dept order by all) as rank +from values('Lisa', 'Sales', 10000, 35) as T(name, dept, salary, age) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`all`", + "proposal" : "`T`.`age`, `T`.`name`, `T`.`dept`, `T`.`salary`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 60, + "stopIndex" : 62, + "fragment" : "all" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-nulls-ordering.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-nulls-ordering.sql.out new file mode 100644 index 0000000000000..61c9c9578ddd8 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-nulls-ordering.sql.out @@ -0,0 +1,184 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create table spark_10747(col1 int, col2 int, col3 int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`spark_10747`, false + + +-- !query +INSERT INTO spark_10747 VALUES (6, 12, 10), (6, 11, 4), (6, 9, 10), (6, 15, 8), +(6, 15, 8), (6, 7, 4), (6, 7, 8), (6, 13, null), (6, 10, null) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/spark_10747, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/spark_10747], Append, `spark_catalog`.`default`.`spark_10747`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/spark_10747), [col1, col2, col3] ++- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 desc nulls last, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query analysis +Sort [sum_col2#xL ASC NULLS FIRST], true ++- Project [col1#x, col2#x, col3#x, sum_col2#xL] + +- Project [col1#x, col2#x, col3#x, sum_col2#xL, sum_col2#xL] + +- Window [sum(col2#x) windowspecdefinition(col1#x, col3#x DESC NULLS LAST, col2#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, 2)) AS sum_col2#xL], [col1#x], [col3#x DESC NULLS LAST, col2#x ASC NULLS FIRST] + +- Project [col1#x, col2#x, col3#x] + +- Filter (col1#x = 6) + +- SubqueryAlias spark_catalog.default.spark_10747 + +- Relation spark_catalog.default.spark_10747[col1#x,col2#x,col3#x] parquet + + +-- !query +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 desc nulls first, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query analysis +Sort [sum_col2#xL ASC NULLS FIRST], true ++- Project [col1#x, col2#x, col3#x, sum_col2#xL] + +- Project [col1#x, col2#x, col3#x, sum_col2#xL, sum_col2#xL] + +- Window [sum(col2#x) windowspecdefinition(col1#x, col3#x DESC NULLS FIRST, col2#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, 2)) AS sum_col2#xL], [col1#x], [col3#x DESC NULLS FIRST, col2#x ASC NULLS FIRST] + +- Project [col1#x, col2#x, col3#x] + +- Filter (col1#x = 6) + +- SubqueryAlias spark_catalog.default.spark_10747 + +- Relation spark_catalog.default.spark_10747[col1#x,col2#x,col3#x] parquet + + +-- !query +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 asc nulls last, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query analysis +Sort [sum_col2#xL ASC NULLS FIRST], true ++- Project [col1#x, col2#x, col3#x, sum_col2#xL] + +- Project [col1#x, col2#x, col3#x, sum_col2#xL, sum_col2#xL] + +- Window [sum(col2#x) windowspecdefinition(col1#x, col3#x ASC NULLS LAST, col2#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, 2)) AS sum_col2#xL], [col1#x], [col3#x ASC NULLS LAST, col2#x ASC NULLS FIRST] + +- Project [col1#x, col2#x, col3#x] + +- Filter (col1#x = 6) + +- SubqueryAlias spark_catalog.default.spark_10747 + +- Relation spark_catalog.default.spark_10747[col1#x,col2#x,col3#x] parquet + + +-- !query +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 asc nulls first, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query analysis +Sort [sum_col2#xL ASC NULLS FIRST], true ++- Project [col1#x, col2#x, col3#x, sum_col2#xL] + +- Project [col1#x, col2#x, col3#x, sum_col2#xL, sum_col2#xL] + +- Window [sum(col2#x) windowspecdefinition(col1#x, col3#x ASC NULLS FIRST, col2#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, 2)) AS sum_col2#xL], [col1#x], [col3#x ASC NULLS FIRST, col2#x ASC NULLS FIRST] + +- Project [col1#x, col2#x, col3#x] + +- Filter (col1#x = 6) + +- SubqueryAlias spark_catalog.default.spark_10747 + +- Relation spark_catalog.default.spark_10747[col1#x,col2#x,col3#x] parquet + + +-- !query +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 ASC NULLS FIRST, COL2 +-- !query analysis +Sort [COL3#x ASC NULLS FIRST, COL2#x ASC NULLS FIRST], true ++- Project [COL1#x, COL2#x, COL3#x] + +- SubqueryAlias spark_catalog.default.spark_10747 + +- Relation spark_catalog.default.spark_10747[col1#x,col2#x,col3#x] parquet + + +-- !query +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 NULLS LAST, COL2 +-- !query analysis +Sort [COL3#x ASC NULLS LAST, COL2#x ASC NULLS FIRST], true ++- Project [COL1#x, COL2#x, COL3#x] + +- SubqueryAlias spark_catalog.default.spark_10747 + +- Relation spark_catalog.default.spark_10747[col1#x,col2#x,col3#x] parquet + + +-- !query +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS FIRST, COL2 +-- !query analysis +Sort [COL3#x DESC NULLS FIRST, COL2#x ASC NULLS FIRST], true ++- Project [COL1#x, COL2#x, COL3#x] + +- SubqueryAlias spark_catalog.default.spark_10747 + +- Relation spark_catalog.default.spark_10747[col1#x,col2#x,col3#x] parquet + + +-- !query +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS LAST, COL2 +-- !query analysis +Sort [COL3#x DESC NULLS LAST, COL2#x ASC NULLS FIRST], true ++- Project [COL1#x, COL2#x, COL3#x] + +- SubqueryAlias spark_catalog.default.spark_10747 + +- Relation spark_catalog.default.spark_10747[col1#x,col2#x,col3#x] parquet + + +-- !query +drop table spark_10747 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`spark_10747`, false, false, false + + +-- !query +create table spark_10747_mix( +col1 string, +col2 int, +col3 double, +col4 decimal(10,2), +col5 decimal(20,1)) +using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`spark_10747_mix`, false + + +-- !query +INSERT INTO spark_10747_mix VALUES +('b', 2, 1.0, 1.00, 10.0), +('d', 3, 2.0, 3.00, 0.0), +('c', 3, 2.0, 2.00, 15.1), +('d', 3, 0.0, 3.00, 1.0), +(null, 3, 0.0, 3.00, 1.0), +('d', 3, null, 4.00, 1.0), +('a', 1, 1.0, 1.00, null), +('c', 3, 2.0, 2.00, null) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/spark_10747_mix, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/spark_10747_mix], Append, `spark_catalog`.`default`.`spark_10747_mix`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/spark_10747_mix), [col1, col2, col3, col4, col5] ++- Project [col1#x, col2#x, cast(col3#x as double) AS col3#x, cast(col4#x as decimal(10,2)) AS col4#x, cast(col5#x as decimal(20,1)) AS col5#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x] + + +-- !query +select * from spark_10747_mix order by col1 nulls last, col5 nulls last +-- !query analysis +Sort [col1#x ASC NULLS LAST, col5#x ASC NULLS LAST], true ++- Project [col1#x, col2#x, col3#x, col4#x, col5#x] + +- SubqueryAlias spark_catalog.default.spark_10747_mix + +- Relation spark_catalog.default.spark_10747_mix[col1#x,col2#x,col3#x,col4#x,col5#x] parquet + + +-- !query +select * from spark_10747_mix order by col1 desc nulls first, col5 desc nulls first +-- !query analysis +Sort [col1#x DESC NULLS FIRST, col5#x DESC NULLS FIRST], true ++- Project [col1#x, col2#x, col3#x, col4#x, col5#x] + +- SubqueryAlias spark_catalog.default.spark_10747_mix + +- Relation spark_catalog.default.spark_10747_mix[col1#x,col2#x,col3#x,col4#x,col5#x] parquet + + +-- !query +select * from spark_10747_mix order by col5 desc nulls first, col3 desc nulls last +-- !query analysis +Sort [col5#x DESC NULLS FIRST, col3#x DESC NULLS LAST], true ++- Project [col1#x, col2#x, col3#x, col4#x, col5#x] + +- SubqueryAlias spark_catalog.default.spark_10747_mix + +- Relation spark_catalog.default.spark_10747_mix[col1#x,col2#x,col3#x,col4#x,col5#x] parquet + + +-- !query +drop table spark_10747_mix +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`spark_10747_mix`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-ordinal.sql.out new file mode 100644 index 0000000000000..2e877714f2b51 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-ordinal.sql.out @@ -0,0 +1,182 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b) +-- !query analysis +CreateViewCommand `data`, select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b), false, false, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select * from data order by 1 desc +-- !query analysis +Sort [a#x DESC NULLS LAST], true ++- Project [a#x, b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select * from data order by 1 desc, b desc +-- !query analysis +Sort [a#x DESC NULLS LAST, b#x DESC NULLS LAST], true ++- Project [a#x, b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select * from data order by 1 desc, 2 desc +-- !query analysis +Sort [a#x DESC NULLS LAST, b#x DESC NULLS LAST], true ++- Project [a#x, b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select * from data order by 1 + 0 desc, b desc +-- !query analysis +Sort [(1 + 0) DESC NULLS LAST, b#x DESC NULLS LAST], true ++- Project [a#x, b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select * from data order by 0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "0", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 29, + "fragment" : "0" + } ] +} + + +-- !query +select * from data order by -1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 30, + "fragment" : "-1" + } ] +} + + +-- !query +select * from data order by 3 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "3", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 29, + "fragment" : "3" + } ] +} + + +-- !query +select * from data sort by 1 desc +-- !query analysis +Sort [a#x DESC NULLS LAST], false ++- Project [a#x, b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +set spark.sql.orderByOrdinal=false +-- !query analysis +SetCommand (spark.sql.orderByOrdinal,Some(false)) + + +-- !query +select * from data order by 0 +-- !query analysis +Sort [0 ASC NULLS FIRST], true ++- Project [a#x, b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select * from data sort by 0 +-- !query analysis +Sort [0 ASC NULLS FIRST], false ++- Project [a#x, b#x] + +- SubqueryAlias data + +- View (`data`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias data + +- LocalRelation [a#x, b#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/outer-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/outer-join.sql.out new file mode 100644 index 0000000000000..9c07012a0b455 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/outer-join.sql.out @@ -0,0 +1,100 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(-234), (145), (367), (975), (298) +as t1(int_col1) +-- !query analysis +CreateViewCommand `t1`, SELECT * FROM VALUES +(-234), (145), (367), (975), (298) +as t1(int_col1), false, true, LocalTempView, true + +- Project [int_col1#x] + +- SubqueryAlias t1 + +- LocalRelation [int_col1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES +(-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) +as t2(int_col0, int_col1) +-- !query analysis +CreateViewCommand `t2`, SELECT * FROM VALUES +(-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) +as t2(int_col0, int_col1), false, true, LocalTempView, true + +- Project [int_col0#x, int_col1#x] + +- SubqueryAlias t2 + +- LocalRelation [int_col0#x, int_col1#x] + + +-- !query +SELECT + (SUM(COALESCE(t1.int_col1, t2.int_col0))), + ((COALESCE(t1.int_col1, t2.int_col0)) * 2) +FROM t1 +RIGHT JOIN t2 + ON (t2.int_col0) = (t1.int_col1) +GROUP BY GREATEST(COALESCE(t2.int_col1, 109), COALESCE(t1.int_col1, -449)), + COALESCE(t1.int_col1, t2.int_col0) +HAVING (SUM(COALESCE(t1.int_col1, t2.int_col0))) + > ((COALESCE(t1.int_col1, t2.int_col0)) * 2) +-- !query analysis +Filter (sum(coalesce(int_col1, int_col0))#xL > cast((coalesce(int_col1, int_col0) * 2)#x as bigint)) ++- Aggregate [greatest(coalesce(int_col1#x, 109), coalesce(int_col1#x, -449)), coalesce(int_col1#x, int_col0#x)], [sum(coalesce(int_col1#x, int_col0#x)) AS sum(coalesce(int_col1, int_col0))#xL, (coalesce(int_col1#x, int_col0#x) * 2) AS (coalesce(int_col1, int_col0) * 2)#x] + +- Join RightOuter, (int_col0#x = int_col1#x) + :- SubqueryAlias t1 + : +- View (`t1`, [int_col1#x]) + : +- Project [cast(int_col1#x as int) AS int_col1#x] + : +- Project [int_col1#x] + : +- SubqueryAlias t1 + : +- LocalRelation [int_col1#x] + +- SubqueryAlias t2 + +- View (`t2`, [int_col0#x,int_col1#x]) + +- Project [cast(int_col0#x as int) AS int_col0#x, cast(int_col1#x as int) AS int_col1#x] + +- Project [int_col0#x, int_col1#x] + +- SubqueryAlias t2 + +- LocalRelation [int_col0#x, int_col1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1) +-- !query analysis +CreateViewCommand `t1`, SELECT * FROM VALUES (97) as t1(int_col1), false, true, LocalTempView, true + +- Project [int_col1#x] + +- SubqueryAlias t1 + +- LocalRelation [int_col1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1) +-- !query analysis +CreateViewCommand `t2`, SELECT * FROM VALUES (0) as t2(int_col1), false, true, LocalTempView, true + +- Project [int_col1#x] + +- SubqueryAlias t2 + +- LocalRelation [int_col1#x] + + +-- !query +SELECT * +FROM ( +SELECT + COALESCE(t2.int_col1, t1.int_col1) AS int_col + FROM t1 + LEFT JOIN t2 ON false +) t where (t.int_col) is not null +-- !query analysis +Project [int_col#x] ++- Filter isnotnull(int_col#x) + +- SubqueryAlias t + +- Project [coalesce(int_col1#x, int_col1#x) AS int_col#x] + +- Join LeftOuter, false + :- SubqueryAlias t1 + : +- View (`t1`, [int_col1#x]) + : +- Project [cast(int_col1#x as int) AS int_col1#x] + : +- Project [int_col1#x] + : +- SubqueryAlias t1 + : +- LocalRelation [int_col1#x] + +- SubqueryAlias t2 + +- View (`t2`, [int_col1#x]) + +- Project [cast(int_col1#x as int) AS int_col1#x] + +- Project [int_col1#x] + +- SubqueryAlias t2 + +- LocalRelation [int_col1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/parse-schema-string.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/parse-schema-string.sql.out new file mode 100644 index 0000000000000..45fc3bd03a782 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/parse-schema-string.sql.out @@ -0,0 +1,27 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select from_csv('1', 'create INT') +-- !query analysis +Project [from_csv(StructField(create,IntegerType,true), 1, Some(America/Los_Angeles), None) AS from_csv(1)#x] ++- OneRowRelation + + +-- !query +select from_csv('1', 'cube INT') +-- !query analysis +Project [from_csv(StructField(cube,IntegerType,true), 1, Some(America/Los_Angeles), None) AS from_csv(1)#x] ++- OneRowRelation + + +-- !query +select from_json('{"create":1}', 'create INT') +-- !query analysis +Project [from_json(StructField(create,IntegerType,true), {"create":1}, Some(America/Los_Angeles)) AS from_json({"create":1})#x] ++- OneRowRelation + + +-- !query +select from_json('{"cube":1}', 'cube INT') +-- !query analysis +Project [from_json(StructField(cube,IntegerType,true), {"cube":1}, Some(America/Los_Angeles)) AS from_json({"cube":1})#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out new file mode 100644 index 0000000000000..f4c05039b7f6b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out @@ -0,0 +1,762 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW aggr AS SELECT * FROM VALUES +(0, 0), (0, 10), (0, 20), (0, 30), (0, 40), (1, 10), (1, 20), (2, 10), (2, 20), (2, 25), (2, 30), (3, 60), (4, null) +AS aggr(k, v) +-- !query analysis +CreateViewCommand `aggr`, SELECT * FROM VALUES +(0, 0), (0, 10), (0, 20), (0, 30), (0, 40), (1, 10), (1, 20), (2, 10), (2, 20), (2, 25), (2, 30), (3, 60), (4, null) +AS aggr(k, v), false, true, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias aggr + +- LocalRelation [k#x, v#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW basic_pays AS SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary) +-- !query analysis +CreateViewCommand `basic_pays`, SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary), false, true, LocalTempView, true + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- LocalRelation [employee_name#x, department#x, salary#x] + + +-- !query +SELECT + percentile_cont(0.25) WITHIN GROUP (ORDER BY v), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +-- !query analysis +Aggregate [percentile_cont(v#x, cast(0.25 as double), false) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_cont(v#x, cast(0.25 as double), false) FILTER (WHERE (k#x > 0)) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE (k > 0))#x, percentile_cont(v#x, cast(0.25 as double), true) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC)#x, percentile_cont(v#x, cast(0.25 as double), true) FILTER (WHERE (k#x > 0)) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE (k > 0))#x] ++- SubqueryAlias aggr + +- View (`aggr`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias aggr + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY v), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +GROUP BY k +ORDER BY k +-- !query analysis +Sort [k#x ASC NULLS FIRST], true ++- Aggregate [k#x], [k#x, percentile_cont(v#x, cast(0.25 as double), false) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_cont(v#x, cast(0.25 as double), false) FILTER (WHERE (k#x > 0)) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE (k > 0))#x, percentile_cont(v#x, cast(0.25 as double), true) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC)#x, percentile_cont(v#x, cast(0.25 as double), true) FILTER (WHERE (k#x > 0)) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE (k > 0))#x] + +- SubqueryAlias aggr + +- View (`aggr`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias aggr + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT + percentile_disc(0.25) WITHIN GROUP (ORDER BY v), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +-- !query analysis +Aggregate [percentile_disc(v#x, cast(0.25 as double), false, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_disc(v#x, cast(0.25 as double), false, 0, 0) FILTER (WHERE (k#x > 0)) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE (k > 0))#x, percentile_disc(v#x, cast(0.25 as double), true, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC)#x, percentile_disc(v#x, cast(0.25 as double), true, 0, 0) FILTER (WHERE (k#x > 0)) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE (k > 0))#x] ++- SubqueryAlias aggr + +- View (`aggr`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias aggr + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY v), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +GROUP BY k +ORDER BY k +-- !query analysis +Sort [k#x ASC NULLS FIRST], true ++- Aggregate [k#x], [k#x, percentile_disc(v#x, cast(0.25 as double), false, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_disc(v#x, cast(0.25 as double), false, 0, 0) FILTER (WHERE (k#x > 0)) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE (k > 0))#x, percentile_disc(v#x, cast(0.25 as double), true, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC)#x, percentile_disc(v#x, cast(0.25 as double), true, 0, 0) FILTER (WHERE (k#x > 0)) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE (k > 0))#x] + +- SubqueryAlias aggr + +- View (`aggr`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias aggr + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT + median(v), + percentile(v, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY v) +FROM aggr +-- !query analysis +Aggregate [median(v#x) AS median(v)#x, percentile(v#x, cast(0.5 as double), 1, 0, 0, false) AS percentile(v, 0.5, 1)#x, percentile_cont(v#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x] ++- SubqueryAlias aggr + +- View (`aggr`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias aggr + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT + k, + median(v), + percentile(v, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY v) +FROM aggr +GROUP BY k +ORDER BY k +-- !query analysis +Sort [k#x ASC NULLS FIRST], true ++- Aggregate [k#x], [k#x, median(v#x) AS median(v)#x, percentile(v#x, cast(0.5 as double), 1, 0, 0, false) AS percentile(v, 0.5, 1)#x, percentile_cont(v#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x] + +- SubqueryAlias aggr + +- View (`aggr`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias aggr + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department), + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department) +FROM basic_pays +ORDER BY salary +-- !query analysis +Sort [salary#x ASC NULLS FIRST], true ++- Project [employee_name#x, department#x, salary#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Project [employee_name#x, department#x, salary#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Window [percentile_cont(salary#x, cast(0.25 as double), false) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(salary#x, cast(0.25 as double), false, 0, 0) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(salary#x, cast(0.25 as double), true) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(salary#x, cast(0.25 as double), true, 0, 0) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x], [department#x] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- View (`basic_pays`, [employee_name#x,department#x,salary#x]) + +- Project [cast(employee_name#x as string) AS employee_name#x, cast(department#x as string) AS department#x, cast(salary#x as int) AS salary#x] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- LocalRelation [employee_name#x, department#x, salary#x] + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary), + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2411", + "messageParameters" : { + "aggFunc" : "percentile_cont" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 157, + "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2411", + "messageParameters" : { + "aggFunc" : "percentile_disc" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 157, + "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + median(salary) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2411", + "messageParameters" : { + "aggFunc" : "median" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 119, + "fragment" : "median(salary) OVER (PARTITION BY department ORDER BY salary)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2411", + "messageParameters" : { + "aggFunc" : "percentile_cont" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 190, + "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2411", + "messageParameters" : { + "aggFunc" : "percentile_disc" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 190, + "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2411", + "messageParameters" : { + "aggFunc" : "median" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 152, + "fragment" : "median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department) +ORDER BY salary +-- !query analysis +Sort [salary#x ASC NULLS FIRST], true ++- Project [employee_name#x, department#x, salary#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Project [employee_name#x, department#x, salary#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Window [percentile_cont(salary#x, cast(0.25 as double), false) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(salary#x, cast(0.25 as double), false, 0, 0) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(salary#x, cast(0.25 as double), true) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(salary#x, cast(0.25 as double), true, 0, 0) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x], [department#x] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- View (`basic_pays`, [employee_name#x,department#x,salary#x]) + +- Project [cast(employee_name#x as string) AS employee_name#x, cast(department#x as string) AS department#x, cast(salary#x as int) AS salary#x] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- LocalRelation [employee_name#x, department#x, salary#x] + + +-- !query +SELECT + employee_name, + department, + salary, + median(salary) OVER w, + percentile_cont(0.5) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.5) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.5) WITHIN GROUP (ORDER BY salary DESC) OVER w, + percentile_disc(0.5) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WHERE salary > 8900 +WINDOW w AS (PARTITION BY department) +ORDER BY salary +-- !query analysis +Sort [salary#x ASC NULLS FIRST], true ++- Project [employee_name#x, department#x, salary#x, median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.5) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.5) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Project [employee_name#x, department#x, salary#x, median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.5) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.5) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.5) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.5) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Window [median(salary#x) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(salary#x, cast(0.5 as double), false) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(salary#x, cast(0.5 as double), false, 0, 0) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_disc(0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(salary#x, cast(0.5 as double), true) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(salary#x, cast(0.5 as double), true, 0, 0) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_disc(0.5) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x], [department#x] + +- Project [employee_name#x, department#x, salary#x] + +- Filter (salary#x > 8900) + +- SubqueryAlias basic_pays + +- View (`basic_pays`, [employee_name#x,department#x,salary#x]) + +- Project [cast(employee_name#x as string) AS employee_name#x, cast(department#x as string) AS department#x, cast(salary#x as int) AS salary#x] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- LocalRelation [employee_name#x, department#x, salary#x] + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ORDER BY salary) +ORDER BY salary +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2411", + "messageParameters" : { + "aggFunc" : "percentile_cont" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 117, + "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ORDER BY salary) +ORDER BY salary +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2411", + "messageParameters" : { + "aggFunc" : "percentile_disc" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 117, + "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + median(salary) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ORDER BY salary) +ORDER BY salary +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2411", + "messageParameters" : { + "aggFunc" : "median" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 79, + "fragment" : "median(salary) OVER w" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2411", + "messageParameters" : { + "aggFunc" : "percentile_cont" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 117, + "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2411", + "messageParameters" : { + "aggFunc" : "percentile_disc" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 117, + "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + median(salary) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2411", + "messageParameters" : { + "aggFunc" : "median" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 79, + "fragment" : "median(salary) OVER w" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW intervals AS SELECT * FROM VALUES +(0, INTERVAL '0' MONTH, INTERVAL '0' SECOND, INTERVAL '0' MINUTE), +(0, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(0, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(0, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(0, INTERVAL '40' MONTH, INTERVAL '40' SECOND, INTERVAL '40' MINUTE), +(1, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(1, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(2, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '25' MONTH, INTERVAL '25' SECOND, INTERVAL '25' MINUTE), +(2, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(3, INTERVAL '60' MONTH, INTERVAL '60' SECOND, INTERVAL '60' MINUTE), +(4, null, null, null) +AS intervals(k, dt, ym, dt2) +-- !query analysis +CreateViewCommand `intervals`, SELECT * FROM VALUES +(0, INTERVAL '0' MONTH, INTERVAL '0' SECOND, INTERVAL '0' MINUTE), +(0, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(0, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(0, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(0, INTERVAL '40' MONTH, INTERVAL '40' SECOND, INTERVAL '40' MINUTE), +(1, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(1, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(2, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '25' MONTH, INTERVAL '25' SECOND, INTERVAL '25' MINUTE), +(2, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(3, INTERVAL '60' MONTH, INTERVAL '60' SECOND, INTERVAL '60' MINUTE), +(4, null, null, null) +AS intervals(k, dt, ym, dt2), false, true, LocalTempView, true + +- Project [k#x, dt#x, ym#x, dt2#x] + +- SubqueryAlias intervals + +- LocalRelation [k#x, dt#x, ym#x, dt2#x] + + +-- !query +SELECT + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt), + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt DESC) +FROM intervals +-- !query analysis +Aggregate [percentile_cont(dt#x, cast(0.25 as double), false) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_cont(dt#x, cast(0.25 as double), true) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC)#x] ++- SubqueryAlias intervals + +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x]) + +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x] + +- Project [k#x, dt#x, ym#x, dt2#x] + +- SubqueryAlias intervals + +- LocalRelation [k#x, dt#x, ym#x, dt2#x] + + +-- !query +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY ym), + percentile_cont(0.25) WITHIN GROUP (ORDER BY ym DESC) +FROM intervals +GROUP BY k +ORDER BY k +-- !query analysis +Sort [k#x ASC NULLS FIRST], true ++- Aggregate [k#x], [k#x, percentile_cont(ym#x, cast(0.25 as double), false) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_cont(ym#x, cast(0.25 as double), true) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC)#x] + +- SubqueryAlias intervals + +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x]) + +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x] + +- Project [k#x, dt#x, ym#x, dt2#x] + +- SubqueryAlias intervals + +- LocalRelation [k#x, dt#x, ym#x, dt2#x] + + +-- !query +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2), + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2 DESC) +FROM intervals +GROUP BY k +ORDER BY k +-- !query analysis +Sort [k#x ASC NULLS FIRST], true ++- Aggregate [k#x], [k#x, percentile_cont(dt2#x, cast(0.25 as double), false) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_cont(dt2#x, cast(0.25 as double), true) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC)#x] + +- SubqueryAlias intervals + +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x]) + +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x] + +- Project [k#x, dt#x, ym#x, dt2#x] + +- SubqueryAlias intervals + +- LocalRelation [k#x, dt#x, ym#x, dt2#x] + + +-- !query +SELECT + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt), + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt DESC) +FROM intervals +-- !query analysis +Aggregate [percentile_disc(dt#x, cast(0.25 as double), false, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_disc(dt#x, cast(0.25 as double), true, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC)#x] ++- SubqueryAlias intervals + +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x]) + +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x] + +- Project [k#x, dt#x, ym#x, dt2#x] + +- SubqueryAlias intervals + +- LocalRelation [k#x, dt#x, ym#x, dt2#x] + + +-- !query +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY ym), + percentile_disc(0.25) WITHIN GROUP (ORDER BY ym DESC) +FROM intervals +GROUP BY k +ORDER BY k +-- !query analysis +Sort [k#x ASC NULLS FIRST], true ++- Aggregate [k#x], [k#x, percentile_disc(ym#x, cast(0.25 as double), false, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_disc(ym#x, cast(0.25 as double), true, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC)#x] + +- SubqueryAlias intervals + +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x]) + +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x] + +- Project [k#x, dt#x, ym#x, dt2#x] + +- SubqueryAlias intervals + +- LocalRelation [k#x, dt#x, ym#x, dt2#x] + + +-- !query +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2), + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2 DESC) +FROM intervals +GROUP BY k +ORDER BY k +-- !query analysis +Sort [k#x ASC NULLS FIRST], true ++- Aggregate [k#x], [k#x, percentile_disc(dt2#x, cast(0.25 as double), false, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_disc(dt2#x, cast(0.25 as double), true, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC)#x] + +- SubqueryAlias intervals + +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x]) + +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x] + +- Project [k#x, dt#x, ym#x, dt2#x] + +- SubqueryAlias intervals + +- LocalRelation [k#x, dt#x, ym#x, dt2#x] + + +-- !query +SELECT + median(dt), + percentile(dt, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY dt) +FROM intervals +-- !query analysis +Aggregate [median(dt#x) AS median(dt)#x, percentile(dt#x, cast(0.5 as double), 1, 0, 0, false) AS percentile(dt, 0.5, 1)#x, percentile_cont(dt#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x] ++- SubqueryAlias intervals + +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x]) + +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x] + +- Project [k#x, dt#x, ym#x, dt2#x] + +- SubqueryAlias intervals + +- LocalRelation [k#x, dt#x, ym#x, dt2#x] + + +-- !query +SELECT + k, + median(ym), + percentile(ym, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY ym) +FROM intervals +GROUP BY k +ORDER BY k +-- !query analysis +Sort [k#x ASC NULLS FIRST], true ++- Aggregate [k#x], [k#x, median(ym#x) AS median(ym)#x, percentile(ym#x, cast(0.5 as double), 1, 0, 0, false) AS percentile(ym, 0.5, 1)#x, percentile_cont(ym#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x] + +- SubqueryAlias intervals + +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x]) + +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x] + +- Project [k#x, dt#x, ym#x, dt2#x] + +- SubqueryAlias intervals + +- LocalRelation [k#x, dt#x, ym#x, dt2#x] + + +-- !query +SELECT + k, + median(dt2), + percentile(dt2, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY dt2) +FROM intervals +GROUP BY k +ORDER BY k +-- !query analysis +Sort [k#x ASC NULLS FIRST], true ++- Aggregate [k#x], [k#x, median(dt2#x) AS median(dt2)#x, percentile(dt2#x, cast(0.5 as double), 1, 0, 0, false) AS percentile(dt2, 0.5, 1)#x, percentile_cont(dt2#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x] + +- SubqueryAlias intervals + +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x]) + +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x] + +- Project [k#x, dt#x, ym#x, dt2#x] + +- SubqueryAlias intervals + +- LocalRelation [k#x, dt#x, ym#x, dt2#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/pivot.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/pivot.sql.out new file mode 100644 index 0000000000000..ef9e4daf70975 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/pivot.sql.out @@ -0,0 +1,793 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view courseSales as select * from values + ("dotNET", 2012, 10000), + ("Java", 2012, 20000), + ("dotNET", 2012, 5000), + ("dotNET", 2013, 48000), + ("Java", 2013, 30000) + as courseSales(course, year, earnings) +-- !query analysis +CreateViewCommand `courseSales`, select * from values + ("dotNET", 2012, 10000), + ("Java", 2012, 20000), + ("dotNET", 2012, 5000), + ("dotNET", 2013, 48000), + ("Java", 2013, 30000) + as courseSales(course, year, earnings), false, false, LocalTempView, true + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +create temporary view years as select * from values + (2012, 1), + (2013, 2) + as years(y, s) +-- !query analysis +CreateViewCommand `years`, select * from values + (2012, 1), + (2013, 2) + as years(y, s), false, false, LocalTempView, true + +- Project [y#x, s#x] + +- SubqueryAlias years + +- LocalRelation [y#x, s#x] + + +-- !query +create temporary view yearsWithComplexTypes as select * from values + (2012, array(1, 1), map('1', 1), struct(1, 'a')), + (2013, array(2, 2), map('2', 2), struct(2, 'b')) + as yearsWithComplexTypes(y, a, m, s) +-- !query analysis +CreateViewCommand `yearsWithComplexTypes`, select * from values + (2012, array(1, 1), map('1', 1), struct(1, 'a')), + (2013, array(2, 2), map('2', 2), struct(2, 'b')) + as yearsWithComplexTypes(y, a, m, s), false, false, LocalTempView, true + +- Project [y#x, a#x, m#x, s#x] + +- SubqueryAlias yearsWithComplexTypes + +- LocalRelation [y#x, a#x, m#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(earnings) + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +Project [year#x, dotNET#xL, Java#xL] ++- Project [year#x, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[0] AS dotNET#xL, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[1] AS Java#xL] + +- Aggregate [year#x], [year#x, pivotfirst(course#x, sum(__auto_generated_subquery_name.earnings)#xL, dotNET, Java, 0, 0) AS __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x] + +- Aggregate [year#x, course#x], [year#x, course#x, sum(earnings#x) AS sum(__auto_generated_subquery_name.earnings)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [year#x, course#x, earnings#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT * FROM courseSales +PIVOT ( + sum(earnings) + FOR year IN (2012, 2013) +) +-- !query analysis +Project [course#x, 2012#xL, 2013#xL] ++- Project [course#x, __pivot_sum(coursesales.earnings) AS `sum(coursesales.earnings)`#x[0] AS 2012#xL, __pivot_sum(coursesales.earnings) AS `sum(coursesales.earnings)`#x[1] AS 2013#xL] + +- Aggregate [course#x], [course#x, pivotfirst(year#x, sum(coursesales.earnings)#xL, 2012, 2013, 0, 0) AS __pivot_sum(coursesales.earnings) AS `sum(coursesales.earnings)`#x] + +- Aggregate [course#x, year#x], [course#x, year#x, sum(earnings#x) AS sum(coursesales.earnings)#xL] + +- 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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(earnings), avg(earnings) + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +Project [year#x, dotNET_sum(earnings)#xL, dotNET_avg(earnings)#x, Java_sum(earnings)#xL, Java_avg(earnings)#x] ++- Project [year#x, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[0] AS dotNET_sum(earnings)#xL, __pivot_avg(__auto_generated_subquery_name.earnings) AS `avg(__auto_generated_subquery_name.earnings)`#x[0] AS dotNET_avg(earnings)#x, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[1] AS Java_sum(earnings)#xL, __pivot_avg(__auto_generated_subquery_name.earnings) AS `avg(__auto_generated_subquery_name.earnings)`#x[1] AS Java_avg(earnings)#x] + +- Aggregate [year#x], [year#x, pivotfirst(course#x, sum(__auto_generated_subquery_name.earnings)#xL, dotNET, Java, 0, 0) AS __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x, pivotfirst(course#x, avg(__auto_generated_subquery_name.earnings)#x, dotNET, Java, 0, 0) AS __pivot_avg(__auto_generated_subquery_name.earnings) AS `avg(__auto_generated_subquery_name.earnings)`#x] + +- Aggregate [year#x, course#x], [year#x, course#x, sum(earnings#x) AS sum(__auto_generated_subquery_name.earnings)#xL, avg(earnings#x) AS avg(__auto_generated_subquery_name.earnings)#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [year#x, course#x, earnings#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT * FROM ( + SELECT course, earnings FROM courseSales +) +PIVOT ( + sum(earnings) + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +Project [dotNET#xL, Java#xL] ++- Project [__pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[0] AS dotNET#xL, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[1] AS Java#xL] + +- Aggregate [pivotfirst(course#x, sum(__auto_generated_subquery_name.earnings)#xL, dotNET, Java, 0, 0) AS __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x] + +- Aggregate [course#x], [course#x, sum(earnings#x) AS sum(__auto_generated_subquery_name.earnings)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, earnings#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(earnings), min(year) + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +Project [dotNET_sum(earnings)#xL, dotNET_min(year)#x, Java_sum(earnings)#xL, Java_min(year)#x] ++- Project [__pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[0] AS dotNET_sum(earnings)#xL, __pivot_min(__auto_generated_subquery_name.year) AS `min(__auto_generated_subquery_name.year)`#x[0] AS dotNET_min(year)#x, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[1] AS Java_sum(earnings)#xL, __pivot_min(__auto_generated_subquery_name.year) AS `min(__auto_generated_subquery_name.year)`#x[1] AS Java_min(year)#x] + +- Aggregate [pivotfirst(course#x, sum(__auto_generated_subquery_name.earnings)#xL, dotNET, Java, 0, 0) AS __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x, pivotfirst(course#x, min(__auto_generated_subquery_name.year)#x, dotNET, Java, 0, 0) AS __pivot_min(__auto_generated_subquery_name.year) AS `min(__auto_generated_subquery_name.year)`#x] + +- Aggregate [course#x], [course#x, sum(earnings#x) AS sum(__auto_generated_subquery_name.earnings)#xL, min(year#x) AS min(__auto_generated_subquery_name.year)#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [year#x, course#x, earnings#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings) + FOR s IN (1, 2) +) +-- !query analysis +Project [course#x, year#x, 1#xL, 2#xL] ++- Project [course#x, year#x, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[0] AS 1#xL, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[1] AS 2#xL] + +- Aggregate [course#x, year#x], [course#x, year#x, pivotfirst(s#x, sum(__auto_generated_subquery_name.earnings)#xL, 1, 2, 0, 0) AS __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x] + +- Aggregate [course#x, year#x, s#x], [course#x, year#x, s#x, sum(earnings#x) AS sum(__auto_generated_subquery_name.earnings)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, year#x, earnings#x, s#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias years + +- View (`years`, [y#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(s#x as int) AS s#x] + +- Project [y#x, s#x] + +- SubqueryAlias years + +- LocalRelation [y#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings), min(s) + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +Project [year#x, dotNET_sum(earnings)#xL, dotNET_min(s)#x, Java_sum(earnings)#xL, Java_min(s)#x] ++- Project [year#x, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[0] AS dotNET_sum(earnings)#xL, __pivot_min(__auto_generated_subquery_name.s) AS `min(__auto_generated_subquery_name.s)`#x[0] AS dotNET_min(s)#x, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[1] AS Java_sum(earnings)#xL, __pivot_min(__auto_generated_subquery_name.s) AS `min(__auto_generated_subquery_name.s)`#x[1] AS Java_min(s)#x] + +- Aggregate [year#x], [year#x, pivotfirst(course#x, sum(__auto_generated_subquery_name.earnings)#xL, dotNET, Java, 0, 0) AS __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x, pivotfirst(course#x, min(__auto_generated_subquery_name.s)#x, dotNET, Java, 0, 0) AS __pivot_min(__auto_generated_subquery_name.s) AS `min(__auto_generated_subquery_name.s)`#x] + +- Aggregate [year#x, course#x], [year#x, course#x, sum(earnings#x) AS sum(__auto_generated_subquery_name.earnings)#xL, min(s#x) AS min(__auto_generated_subquery_name.s)#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, year#x, earnings#x, s#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias years + +- View (`years`, [y#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(s#x as int) AS s#x] + +- Project [y#x, s#x] + +- SubqueryAlias years + +- LocalRelation [y#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings * s) + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +Project [year#x, dotNET#xL, Java#xL] ++- Project [year#x, __pivot_sum((__auto_generated_subquery_name.earnings * __auto_generated_subquery_name.s)) AS `sum((__auto_generated_subquery_name.earnings * __auto_generated_subquery_name.s))`#x[0] AS dotNET#xL, __pivot_sum((__auto_generated_subquery_name.earnings * __auto_generated_subquery_name.s)) AS `sum((__auto_generated_subquery_name.earnings * __auto_generated_subquery_name.s))`#x[1] AS Java#xL] + +- Aggregate [year#x], [year#x, pivotfirst(course#x, sum((__auto_generated_subquery_name.earnings * __auto_generated_subquery_name.s))#xL, dotNET, Java, 0, 0) AS __pivot_sum((__auto_generated_subquery_name.earnings * __auto_generated_subquery_name.s)) AS `sum((__auto_generated_subquery_name.earnings * __auto_generated_subquery_name.s))`#x] + +- Aggregate [year#x, course#x], [year#x, course#x, sum((earnings#x * s#x)) AS sum((__auto_generated_subquery_name.earnings * __auto_generated_subquery_name.s))#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, year#x, earnings#x, s#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias years + +- View (`years`, [y#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(s#x as int) AS s#x] + +- Project [y#x, s#x] + +- SubqueryAlias years + +- LocalRelation [y#x, s#x] + + +-- !query +SELECT 2012_s, 2013_s, 2012_a, 2013_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + sum(e) s, avg(e) a + FOR y IN (2012, 2013) +) +-- !query analysis +Project [2012_s#xL, 2013_s#xL, 2012_a#x, 2013_a#x, c#x] ++- Project [c#x, __pivot_sum(__auto_generated_subquery_name.e) AS s AS `sum(__auto_generated_subquery_name.e) AS s`#x[0] AS 2012_s#xL, __pivot_avg(__auto_generated_subquery_name.e) AS a AS `avg(__auto_generated_subquery_name.e) AS a`#x[0] AS 2012_a#x, __pivot_sum(__auto_generated_subquery_name.e) AS s AS `sum(__auto_generated_subquery_name.e) AS s`#x[1] AS 2013_s#xL, __pivot_avg(__auto_generated_subquery_name.e) AS a AS `avg(__auto_generated_subquery_name.e) AS a`#x[1] AS 2013_a#x] + +- Aggregate [c#x], [c#x, pivotfirst(y#x, sum(__auto_generated_subquery_name.e) AS s#xL, 2012, 2013, 0, 0) AS __pivot_sum(__auto_generated_subquery_name.e) AS s AS `sum(__auto_generated_subquery_name.e) AS s`#x, pivotfirst(y#x, avg(__auto_generated_subquery_name.e) AS a#x, 2012, 2013, 0, 0) AS __pivot_avg(__auto_generated_subquery_name.e) AS a AS `avg(__auto_generated_subquery_name.e) AS a`#x] + +- Aggregate [c#x, y#x], [c#x, y#x, sum(e#x) AS sum(__auto_generated_subquery_name.e) AS s#xL, avg(e#x) AS avg(__auto_generated_subquery_name.e) AS a#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [year#x AS y#x, course#x AS c#x, earnings#x 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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT firstYear_s, secondYear_s, firstYear_a, secondYear_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + sum(e) s, avg(e) a + FOR y IN (2012 as firstYear, 2013 secondYear) +) +-- !query analysis +Project [firstYear_s#xL, secondYear_s#xL, firstYear_a#x, secondYear_a#x, c#x] ++- Project [c#x, __pivot_sum(__auto_generated_subquery_name.e) AS s AS `sum(__auto_generated_subquery_name.e) AS s`#x[0] AS firstYear_s#xL, __pivot_avg(__auto_generated_subquery_name.e) AS a AS `avg(__auto_generated_subquery_name.e) AS a`#x[0] AS firstYear_a#x, __pivot_sum(__auto_generated_subquery_name.e) AS s AS `sum(__auto_generated_subquery_name.e) AS s`#x[1] AS secondYear_s#xL, __pivot_avg(__auto_generated_subquery_name.e) AS a AS `avg(__auto_generated_subquery_name.e) AS a`#x[1] AS secondYear_a#x] + +- Aggregate [c#x], [c#x, pivotfirst(y#x, sum(__auto_generated_subquery_name.e) AS s#xL, 2012, 2013, 0, 0) AS __pivot_sum(__auto_generated_subquery_name.e) AS s AS `sum(__auto_generated_subquery_name.e) AS s`#x, pivotfirst(y#x, avg(__auto_generated_subquery_name.e) AS a#x, 2012, 2013, 0, 0) AS __pivot_avg(__auto_generated_subquery_name.e) AS a AS `avg(__auto_generated_subquery_name.e) AS a`#x] + +- Aggregate [c#x, y#x], [c#x, y#x, sum(e#x) AS sum(__auto_generated_subquery_name.e) AS s#xL, avg(e#x) AS avg(__auto_generated_subquery_name.e) AS a#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [year#x AS y#x, course#x AS c#x, earnings#x 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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT * FROM courseSales +PIVOT ( + abs(earnings) + FOR year IN (2012, 2013) +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1006", + "messageParameters" : { + "sql" : "coursesales.earnings" + } +} + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(earnings), year + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1006", + "messageParameters" : { + "sql" : "__auto_generated_subquery_name.year" + } +} + + +-- !query +SELECT * FROM ( + SELECT course, earnings FROM courseSales +) +PIVOT ( + sum(earnings) + FOR year IN (2012, 2013) +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`year`", + "proposal" : "`__auto_generated_subquery_name`.`course`, `__auto_generated_subquery_name`.`earnings`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 113, + "fragment" : "PIVOT (\n sum(earnings)\n FOR year IN (2012, 2013)\n)" + } ] +} + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + ceil(sum(earnings)), avg(earnings) + 1 as a1 + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +Project [year#x, dotNET_CEIL(sum(earnings))#xL, dotNET_a1#x, Java_CEIL(sum(earnings))#xL, Java_a1#x] ++- Project [year#x, __pivot_CEIL(sum(__auto_generated_subquery_name.earnings)) AS `CEIL(sum(__auto_generated_subquery_name.earnings))`#x[0] AS dotNET_CEIL(sum(earnings))#xL, __pivot_(avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1 AS `(avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1`#x[0] AS dotNET_a1#x, __pivot_CEIL(sum(__auto_generated_subquery_name.earnings)) AS `CEIL(sum(__auto_generated_subquery_name.earnings))`#x[1] AS Java_CEIL(sum(earnings))#xL, __pivot_(avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1 AS `(avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1`#x[1] AS Java_a1#x] + +- Aggregate [year#x], [year#x, pivotfirst(course#x, CEIL(sum(__auto_generated_subquery_name.earnings))#xL, dotNET, Java, 0, 0) AS __pivot_CEIL(sum(__auto_generated_subquery_name.earnings)) AS `CEIL(sum(__auto_generated_subquery_name.earnings))`#x, pivotfirst(course#x, (avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1#x, dotNET, Java, 0, 0) AS __pivot_(avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1 AS `(avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1`#x] + +- Aggregate [year#x, course#x], [year#x, course#x, CEIL(sum(earnings#x)) AS CEIL(sum(__auto_generated_subquery_name.earnings))#xL, (avg(earnings#x) + cast(1 as double)) AS (avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [year#x, course#x, earnings#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(avg(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NESTED_AGGREGATE_FUNCTION", + "sqlState" : "42607", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 82, + "stopIndex" : 94, + "fragment" : "avg(earnings)" + } ] +} + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) +) +-- !query analysis +Project [s#x, {dotNET, 2012}#xL, {Java, 2013}#xL] ++- Project [s#x, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[0] AS {dotNET, 2012}#xL, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[1] AS {Java, 2013}#xL] + +- Aggregate [s#x], [s#x, pivotfirst(__pivot_col#x, sum(__auto_generated_subquery_name.earnings)#xL, [dotNET,2012], [Java,2013], 0, 0) AS __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x] + +- Aggregate [s#x, named_struct(course, course#x, year, year#x)], [s#x, named_struct(course, course#x, year, year#x) AS __pivot_col#x, sum(earnings#x) AS sum(__auto_generated_subquery_name.earnings)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, year#x, earnings#x, s#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias years + +- View (`years`, [y#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(s#x as int) AS s#x] + +- Project [y#x, s#x] + +- SubqueryAlias years + +- LocalRelation [y#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, s) IN (('dotNET', 2) as c1, ('Java', 1) as c2) +) +-- !query analysis +Project [year#x, c1#xL, c2#xL] ++- Project [year#x, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[0] AS c1#xL, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[1] AS c2#xL] + +- Aggregate [year#x], [year#x, pivotfirst(__pivot_col#x, sum(__auto_generated_subquery_name.earnings)#xL, [dotNET,2], [Java,1], 0, 0) AS __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x] + +- Aggregate [year#x, named_struct(course, course#x, s, s#x)], [year#x, named_struct(course, course#x, s, s#x) AS __pivot_col#x, sum(earnings#x) AS sum(__auto_generated_subquery_name.earnings)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, year#x, earnings#x, s#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias years + +- View (`years`, [y#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(s#x as int) AS s#x] + +- Project [y#x, s#x] + +- SubqueryAlias years + +- LocalRelation [y#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, year) IN ('dotNET', 'Java') +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIVOT_VALUE_DATA_TYPE_MISMATCH", + "sqlState" : "42K09", + "messageParameters" : { + "pivotType" : "struct", + "value" : "dotNET", + "valueType" : "string" + } +} + + +-- !query +SELECT * FROM courseSales +PIVOT ( + sum(earnings) + FOR year IN (s, 2013) +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`s`", + "proposal" : "`coursesales`.`year`, `coursesales`.`course`, `coursesales`.`earnings`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 66, + "stopIndex" : 66, + "fragment" : "s" + } ] +} + + +-- !query +SELECT * FROM courseSales +PIVOT ( + sum(earnings) + FOR year IN (course, 2013) +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NON_LITERAL_PIVOT_VALUES", + "sqlState" : "42K08", + "messageParameters" : { + "expression" : "\"course\"" + } +} + + +-- !query +SELECT * FROM ( + SELECT course, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + min(a) + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +Project [year#x, dotNET#x, Java#x] ++- Aggregate [year#x], [year#x, min(if ((course#x <=> cast(dotNET as string))) a#x else cast(null as array)) AS dotNET#x, min(if ((course#x <=> cast(Java as string))) a#x else cast(null as array)) AS Java#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, year#x, a#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias yearswithcomplextypes + +- View (`yearsWithComplexTypes`, [y#x,a#x,m#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(a#x as array) AS a#x, cast(m#x as map) AS m#x, cast(s#x as struct) AS s#x] + +- Project [y#x, a#x, m#x, s#x] + +- SubqueryAlias yearsWithComplexTypes + +- LocalRelation [y#x, a#x, m#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT course, year, y, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + max(a) + FOR (y, course) IN ((2012, 'dotNET'), (2013, 'Java')) +) +-- !query analysis +Project [year#x, {2012, dotNET}#x, {2013, Java}#x] ++- Aggregate [year#x], [year#x, max(if ((named_struct(y, y#x, course, course#x) <=> cast(named_struct(col1, 2012, col2, dotNET) as struct))) a#x else cast(null as array)) AS {2012, dotNET}#x, max(if ((named_struct(y, y#x, course, course#x) <=> cast(named_struct(col1, 2013, col2, Java) as struct))) a#x else cast(null as array)) AS {2013, Java}#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, year#x, y#x, a#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias yearswithcomplextypes + +- View (`yearsWithComplexTypes`, [y#x,a#x,m#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(a#x as array) AS a#x, cast(m#x as map) AS m#x, cast(s#x as struct) AS s#x] + +- Project [y#x, a#x, m#x, s#x] + +- SubqueryAlias yearsWithComplexTypes + +- LocalRelation [y#x, a#x, m#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT earnings, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR a IN (array(1, 1), array(2, 2)) +) +-- !query analysis +Project [year#x, [1, 1]#xL, [2, 2]#xL] ++- Project [year#x, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[0] AS [1, 1]#xL, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[1] AS [2, 2]#xL] + +- Aggregate [year#x], [year#x, pivotfirst(a#x, sum(__auto_generated_subquery_name.earnings)#xL, [1,1], [2,2], 0, 0) AS __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x] + +- Aggregate [year#x, a#x], [year#x, a#x, sum(earnings#x) AS sum(__auto_generated_subquery_name.earnings)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [earnings#x, year#x, a#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias yearswithcomplextypes + +- View (`yearsWithComplexTypes`, [y#x,a#x,m#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(a#x as array) AS a#x, cast(m#x as map) AS m#x, cast(s#x as struct) AS s#x] + +- Project [y#x, a#x, m#x, s#x] + +- SubqueryAlias yearsWithComplexTypes + +- LocalRelation [y#x, a#x, m#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT course, earnings, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) +) +-- !query analysis +Project [year#x, {dotNET, [1, 1]}#xL, {Java, [2, 2]}#xL] ++- Project [year#x, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[0] AS {dotNET, [1, 1]}#xL, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[1] AS {Java, [2, 2]}#xL] + +- Aggregate [year#x], [year#x, pivotfirst(__pivot_col#x, sum(__auto_generated_subquery_name.earnings)#xL, [dotNET,[1,1]], [Java,[2,2]], 0, 0) AS __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x] + +- Aggregate [year#x, named_struct(course, course#x, a, a#x)], [year#x, named_struct(course, course#x, a, a#x) AS __pivot_col#x, sum(earnings#x) AS sum(__auto_generated_subquery_name.earnings)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, earnings#x, year#x, a#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias yearswithcomplextypes + +- View (`yearsWithComplexTypes`, [y#x,a#x,m#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(a#x as array) AS a#x, cast(m#x as map) AS m#x, cast(s#x as struct) AS s#x] + +- Project [y#x, a#x, m#x, s#x] + +- SubqueryAlias yearsWithComplexTypes + +- LocalRelation [y#x, a#x, m#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR s IN ((1, 'a'), (2, 'b')) +) +-- !query analysis +Project [year#x, {1, a}#xL, {2, b}#xL] ++- Project [year#x, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[0] AS {1, a}#xL, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[1] AS {2, b}#xL] + +- Aggregate [year#x], [year#x, pivotfirst(s#x, sum(__auto_generated_subquery_name.earnings)#xL, [1,a], [2,b], 0, 0) AS __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x] + +- Aggregate [year#x, s#x], [year#x, s#x, sum(earnings#x) AS sum(__auto_generated_subquery_name.earnings)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [earnings#x, year#x, s#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias yearswithcomplextypes + +- View (`yearsWithComplexTypes`, [y#x,a#x,m#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(a#x as array) AS a#x, cast(m#x as map) AS m#x, cast(s#x as struct) AS s#x] + +- Project [y#x, a#x, m#x, s#x] + +- SubqueryAlias yearsWithComplexTypes + +- LocalRelation [y#x, a#x, m#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT course, earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) +) +-- !query analysis +Project [year#x, {dotNET, {1, a}}#xL, {Java, {2, b}}#xL] ++- Project [year#x, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[0] AS {dotNET, {1, a}}#xL, __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x[1] AS {Java, {2, b}}#xL] + +- Aggregate [year#x], [year#x, pivotfirst(__pivot_col#x, sum(__auto_generated_subquery_name.earnings)#xL, [dotNET,[1,a]], [Java,[2,b]], 0, 0) AS __pivot_sum(__auto_generated_subquery_name.earnings) AS `sum(__auto_generated_subquery_name.earnings)`#x] + +- Aggregate [year#x, named_struct(course, course#x, s, s#x)], [year#x, named_struct(course, course#x, s, s#x) AS __pivot_col#x, sum(earnings#x) AS sum(__auto_generated_subquery_name.earnings)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, earnings#x, year#x, s#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias yearswithcomplextypes + +- View (`yearsWithComplexTypes`, [y#x,a#x,m#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(a#x as array) AS a#x, cast(m#x as map) AS m#x, cast(s#x as struct) AS s#x] + +- Project [y#x, a#x, m#x, s#x] + +- SubqueryAlias yearsWithComplexTypes + +- LocalRelation [y#x, a#x, m#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR m IN (map('1', 1), map('2', 2)) +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPARABLE_PIVOT_COLUMN", + "sqlState" : "42818", + "messageParameters" : { + "columnName" : "`__auto_generated_subquery_name`.`m`" + } +} + + +-- !query +SELECT * FROM ( + SELECT course, earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, m) IN (('dotNET', map('1', 1)), ('Java', map('2', 2))) +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPARABLE_PIVOT_COLUMN", + "sqlState" : "42818", + "messageParameters" : { + "columnName" : "`named_struct('course', __auto_generated_subquery_name`.`course, 'm', __auto_generated_subquery_name`.`m)`" + } +} + + +-- !query +SELECT * FROM ( + SELECT course, earnings, "a" as a, "z" as z, "b" as b, "y" as y, "c" as c, "x" as x, "d" as d, "w" as w + FROM courseSales +) +PIVOT ( + sum(Earnings) + FOR Course IN ('dotNET', 'Java') +) +-- !query analysis +Project [a#x, z#x, b#x, y#x, c#x, x#x, d#x, w#x, dotNET#xL, Java#xL] ++- Project [a#x, z#x, b#x, y#x, c#x, x#x, d#x, w#x, __pivot_sum(__auto_generated_subquery_name.Earnings) AS `sum(__auto_generated_subquery_name.Earnings)`#x[0] AS dotNET#xL, __pivot_sum(__auto_generated_subquery_name.Earnings) AS `sum(__auto_generated_subquery_name.Earnings)`#x[1] AS Java#xL] + +- Aggregate [a#x, z#x, b#x, y#x, c#x, x#x, d#x, w#x], [a#x, z#x, b#x, y#x, c#x, x#x, d#x, w#x, pivotfirst(Course#x, sum(__auto_generated_subquery_name.Earnings)#xL, dotNET, Java, 0, 0) AS __pivot_sum(__auto_generated_subquery_name.Earnings) AS `sum(__auto_generated_subquery_name.Earnings)`#x] + +- Aggregate [a#x, z#x, b#x, y#x, c#x, x#x, d#x, w#x, Course#x], [a#x, z#x, b#x, y#x, c#x, x#x, d#x, w#x, Course#x, sum(Earnings#x) AS sum(__auto_generated_subquery_name.Earnings)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, earnings#x, a AS a#x, z AS z#x, b AS b#x, y AS y#x, c AS c#x, x AS x#x, d AS d#x, w AS w#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part1.sql.out new file mode 100644 index 0000000000000..d582c722ac10b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part1.sql.out @@ -0,0 +1,521 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT avg(four) AS avg_1 FROM onek +-- !query analysis +Aggregate [avg(four#x) AS avg_1#x] ++- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT avg(a) AS avg_32 FROM aggtest WHERE a < 100 +-- !query analysis +Aggregate [avg(a#x) AS avg_32#x] ++- Filter (a#x < 100) + +- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +select CAST(avg(b) AS Decimal(10,3)) AS avg_107_943 FROM aggtest +-- !query analysis +Aggregate [cast(avg(b#x) as decimal(10,3)) AS avg_107_943#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT sum(four) AS sum_1500 FROM onek +-- !query analysis +Aggregate [sum(four#x) AS sum_1500#xL] ++- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT sum(a) AS sum_198 FROM aggtest +-- !query analysis +Aggregate [sum(a#x) AS sum_198#xL] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT sum(b) AS avg_431_773 FROM aggtest +-- !query analysis +Aggregate [sum(b#x) AS avg_431_773#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT max(four) AS max_3 FROM onek +-- !query analysis +Aggregate [max(four#x) AS max_3#x] ++- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT max(a) AS max_100 FROM aggtest +-- !query analysis +Aggregate [max(a#x) AS max_100#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT max(aggtest.b) AS max_324_78 FROM aggtest +-- !query analysis +Aggregate [max(b#x) AS max_324_78#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT stddev_pop(b) FROM aggtest +-- !query analysis +Aggregate [stddev_pop(cast(b#x as double)) AS stddev_pop(b)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT stddev_samp(b) FROM aggtest +-- !query analysis +Aggregate [stddev_samp(cast(b#x as double)) AS stddev_samp(b)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT var_pop(b) FROM aggtest +-- !query analysis +Aggregate [var_pop(cast(b#x as double)) AS var_pop(b)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT var_samp(b) FROM aggtest +-- !query analysis +Aggregate [var_samp(cast(b#x as double)) AS var_samp(b)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT stddev_pop(CAST(b AS Decimal(38,0))) FROM aggtest +-- !query analysis +Aggregate [stddev_pop(cast(cast(b#x as decimal(38,0)) as double)) AS stddev_pop(CAST(b AS DECIMAL(38,0)))#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT stddev_samp(CAST(b AS Decimal(38,0))) FROM aggtest +-- !query analysis +Aggregate [stddev_samp(cast(cast(b#x as decimal(38,0)) as double)) AS stddev_samp(CAST(b AS DECIMAL(38,0)))#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT var_pop(CAST(b AS Decimal(38,0))) FROM aggtest +-- !query analysis +Aggregate [var_pop(cast(cast(b#x as decimal(38,0)) as double)) AS var_pop(CAST(b AS DECIMAL(38,0)))#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT var_samp(CAST(b AS Decimal(38,0))) FROM aggtest +-- !query analysis +Aggregate [var_samp(cast(cast(b#x as decimal(38,0)) as double)) AS var_samp(CAST(b AS DECIMAL(38,0)))#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT var_pop(1.0), var_samp(2.0) +-- !query analysis +Aggregate [var_pop(cast(1.0 as double)) AS var_pop(1.0)#x, var_samp(cast(2.0 as double)) AS var_samp(2.0)#x] ++- OneRowRelation + + +-- !query +SELECT stddev_pop(CAST(3.0 AS Decimal(38,0))), stddev_samp(CAST(4.0 AS Decimal(38,0))) +-- !query analysis +Aggregate [stddev_pop(cast(cast(3.0 as decimal(38,0)) as double)) AS stddev_pop(CAST(3.0 AS DECIMAL(38,0)))#x, stddev_samp(cast(cast(4.0 as decimal(38,0)) as double)) AS stddev_samp(CAST(4.0 AS DECIMAL(38,0)))#x] ++- OneRowRelation + + +-- !query +select sum(CAST(null AS int)) from range(1,4) +-- !query analysis +Aggregate [sum(cast(null as int)) AS sum(CAST(NULL AS INT))#xL] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select sum(CAST(null AS long)) from range(1,4) +-- !query analysis +Aggregate [sum(cast(null as bigint)) AS sum(CAST(NULL AS BIGINT))#xL] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select sum(CAST(null AS Decimal(38,0))) from range(1,4) +-- !query analysis +Aggregate [sum(cast(null as decimal(38,0))) AS sum(CAST(NULL AS DECIMAL(38,0)))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select sum(CAST(null AS DOUBLE)) from range(1,4) +-- !query analysis +Aggregate [sum(cast(null as double)) AS sum(CAST(NULL AS DOUBLE))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select avg(CAST(null AS int)) from range(1,4) +-- !query analysis +Aggregate [avg(cast(null as int)) AS avg(CAST(NULL AS INT))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select avg(CAST(null AS long)) from range(1,4) +-- !query analysis +Aggregate [avg(cast(null as bigint)) AS avg(CAST(NULL AS BIGINT))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select avg(CAST(null AS Decimal(38,0))) from range(1,4) +-- !query analysis +Aggregate [avg(cast(null as decimal(38,0))) AS avg(CAST(NULL AS DECIMAL(38,0)))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select avg(CAST(null AS DOUBLE)) from range(1,4) +-- !query analysis +Aggregate [avg(cast(null as double)) AS avg(CAST(NULL AS DOUBLE))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select sum(CAST('NaN' AS DOUBLE)) from range(1,4) +-- !query analysis +Aggregate [sum(cast(NaN as double)) AS sum(CAST(NaN AS DOUBLE))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select avg(CAST('NaN' AS DOUBLE)) from range(1,4) +-- !query analysis +Aggregate [avg(cast(NaN as double)) AS avg(CAST(NaN AS DOUBLE))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES (CAST('1' AS DOUBLE)), (CAST('infinity' AS DOUBLE))) v(x) +-- !query analysis +Aggregate [avg(cast(x#x as double)) AS avg(CAST(x AS DOUBLE))#x, var_pop(cast(x#x as double)) AS var_pop(CAST(x AS DOUBLE))#x] ++- SubqueryAlias v + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES ('infinity'), ('1')) v(x) +-- !query analysis +Aggregate [avg(cast(x#x as double)) AS avg(CAST(x AS DOUBLE))#x, var_pop(cast(x#x as double)) AS var_pop(CAST(x AS DOUBLE))#x] ++- SubqueryAlias v + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES ('infinity'), ('infinity')) v(x) +-- !query analysis +Aggregate [avg(cast(x#x as double)) AS avg(CAST(x AS DOUBLE))#x, var_pop(cast(x#x as double)) AS var_pop(CAST(x AS DOUBLE))#x] ++- SubqueryAlias v + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES ('-infinity'), ('infinity')) v(x) +-- !query analysis +Aggregate [avg(cast(x#x as double)) AS avg(CAST(x AS DOUBLE))#x, var_pop(cast(x#x as double)) AS var_pop(CAST(x AS DOUBLE))#x] ++- SubqueryAlias v + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x) +-- !query analysis +Aggregate [avg(cast(x#x as double)) AS avg(CAST(x AS DOUBLE))#x, var_pop(cast(x#x as double)) AS var_pop(CAST(x AS DOUBLE))#x] ++- SubqueryAlias v + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES (7000000000005), (7000000000007)) v(x) +-- !query analysis +Aggregate [avg(cast(x#xL as double)) AS avg(CAST(x AS DOUBLE))#x, var_pop(cast(x#xL as double)) AS var_pop(CAST(x AS DOUBLE))#x] ++- SubqueryAlias v + +- Project [col1#xL AS x#xL] + +- LocalRelation [col1#xL] + + +-- !query +SELECT regr_count(b, a) FROM aggtest +-- !query analysis +Aggregate [regr_count(b#x, a#x) AS regr_count(b, a)#xL] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT regr_sxx(b, a) FROM aggtest +-- !query analysis +Aggregate [regr_sxx(cast(b#x as double), cast(a#x as double)) AS regr_sxx(b, a)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT regr_syy(b, a) FROM aggtest +-- !query analysis +Aggregate [regr_syy(cast(b#x as double), cast(a#x as double)) AS regr_syy(b, a)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT regr_sxy(b, a) FROM aggtest +-- !query analysis +Aggregate [regr_sxy(cast(b#x as double), cast(a#x as double)) AS regr_sxy(b, a)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT regr_avgx(b, a), regr_avgy(b, a) FROM aggtest +-- !query analysis +Aggregate [regr_avgx(b#x, a#x) AS regr_avgx(b, a)#x, regr_avgy(b#x, a#x) AS regr_avgy(b, a)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT regr_r2(b, a) FROM aggtest +-- !query analysis +Aggregate [regr_r2(cast(b#x as double), cast(a#x as double)) AS regr_r2(b, a)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT regr_slope(b, a), regr_intercept(b, a) FROM aggtest +-- !query analysis +Aggregate [regr_slope(cast(b#x as double), cast(a#x as double)) AS regr_slope(b, a)#x, regr_intercept(cast(b#x as double), cast(a#x as double)) AS regr_intercept(b, a)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT covar_pop(b, a), covar_samp(b, a) FROM aggtest +-- !query analysis +Aggregate [covar_pop(cast(b#x as double), cast(a#x as double)) AS covar_pop(b, a)#x, covar_samp(cast(b#x as double), cast(a#x as double)) AS covar_samp(b, a)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT corr(b, a) FROM aggtest +-- !query analysis +Aggregate [corr(cast(b#x as double), cast(a#x as double)) AS corr(b, a)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +CREATE TEMPORARY VIEW regr_test AS SELECT * FROM VALUES (10,150),(20,250),(30,350),(80,540),(100,200) AS regr_test (x, y) +-- !query analysis +CreateViewCommand `regr_test`, SELECT * FROM VALUES (10,150),(20,250),(30,350),(80,540),(100,200) AS regr_test (x, y), false, false, LocalTempView, true + +- Project [x#x, y#x] + +- SubqueryAlias regr_test + +- LocalRelation [x#x, y#x] + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (10,20,30,80) +-- !query analysis +Aggregate [count(1) AS count(1)#xL, sum(x#x) AS sum(x)#xL, regr_sxx(cast(y#x as double), cast(x#x as double)) AS regr_sxx(y, x)#x, sum(y#x) AS sum(y)#xL, regr_syy(cast(y#x as double), cast(x#x as double)) AS regr_syy(y, x)#x, regr_sxy(cast(y#x as double), cast(x#x as double)) AS regr_sxy(y, x)#x] ++- Filter x#x IN (10,20,30,80) + +- SubqueryAlias regr_test + +- View (`regr_test`, [x#x,y#x]) + +- Project [cast(x#x as int) AS x#x, cast(y#x as int) AS y#x] + +- Project [x#x, y#x] + +- SubqueryAlias regr_test + +- LocalRelation [x#x, y#x] + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test +-- !query analysis +Aggregate [count(1) AS count(1)#xL, sum(x#x) AS sum(x)#xL, regr_sxx(cast(y#x as double), cast(x#x as double)) AS regr_sxx(y, x)#x, sum(y#x) AS sum(y)#xL, regr_syy(cast(y#x as double), cast(x#x as double)) AS regr_syy(y, x)#x, regr_sxy(cast(y#x as double), cast(x#x as double)) AS regr_sxy(y, x)#x] ++- SubqueryAlias regr_test + +- View (`regr_test`, [x#x,y#x]) + +- Project [cast(x#x as int) AS x#x, cast(y#x as int) AS y#x] + +- Project [x#x, y#x] + +- SubqueryAlias regr_test + +- LocalRelation [x#x, y#x] + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (10,20,30) +-- !query analysis +Aggregate [count(1) AS count(1)#xL, sum(x#x) AS sum(x)#xL, regr_sxx(cast(y#x as double), cast(x#x as double)) AS regr_sxx(y, x)#x, sum(y#x) AS sum(y)#xL, regr_syy(cast(y#x as double), cast(x#x as double)) AS regr_syy(y, x)#x, regr_sxy(cast(y#x as double), cast(x#x as double)) AS regr_sxy(y, x)#x] ++- Filter x#x IN (10,20,30) + +- SubqueryAlias regr_test + +- View (`regr_test`, [x#x,y#x]) + +- Project [cast(x#x as int) AS x#x, cast(y#x as int) AS y#x] + +- Project [x#x, y#x] + +- SubqueryAlias regr_test + +- LocalRelation [x#x, y#x] + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (80,100) +-- !query analysis +Aggregate [count(1) AS count(1)#xL, sum(x#x) AS sum(x)#xL, regr_sxx(cast(y#x as double), cast(x#x as double)) AS regr_sxx(y, x)#x, sum(y#x) AS sum(y)#xL, regr_syy(cast(y#x as double), cast(x#x as double)) AS regr_syy(y, x)#x, regr_sxy(cast(y#x as double), cast(x#x as double)) AS regr_sxy(y, x)#x] ++- Filter x#x IN (80,100) + +- SubqueryAlias regr_test + +- View (`regr_test`, [x#x,y#x]) + +- Project [cast(x#x as int) AS x#x, cast(y#x as int) AS y#x] + +- Project [x#x, y#x] + +- SubqueryAlias regr_test + +- LocalRelation [x#x, y#x] + + +-- !query +DROP VIEW regr_test +-- !query analysis +DropTempViewCommand regr_test + + +-- !query +SELECT count(four) AS cnt_1000 FROM onek +-- !query analysis +Aggregate [count(four#x) AS cnt_1000#xL] ++- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT count(DISTINCT four) AS cnt_4 FROM onek +-- !query analysis +Aggregate [count(distinct four#x) AS cnt_4#xL] ++- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select ten, count(*), sum(four) from onek +group by ten order by ten +-- !query analysis +Sort [ten#x ASC NULLS FIRST], true ++- Aggregate [ten#x], [ten#x, count(1) AS count(1)#xL, sum(four#x) AS sum(four)#xL] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select ten, count(four), sum(DISTINCT four) from onek +group by ten order by ten +-- !query analysis +Sort [ten#x ASC NULLS FIRST], true ++- Aggregate [ten#x], [ten#x, count(four#x) AS count(four)#xL, sum(distinct four#x) AS sum(DISTINCT four)#xL] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b where sum(distinct a.four) = b.four) +-- !query analysis +Filter exists#x [sum(DISTINCT four)#xL] +: +- Project [1 AS 1#x] +: +- Filter (outer(sum(DISTINCT four)#xL) = cast(four#x as bigint)) +: +- SubqueryAlias b +: +- SubqueryAlias spark_catalog.default.onek +: +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet ++- Aggregate [ten#x], [ten#x, sum(distinct four#x) AS sum(DISTINCT four)#xL] + +- SubqueryAlias a + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b + where sum(distinct a.four + b.four) = b.four) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "sum(DISTINCT (outer(a.four) + b.four))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 114, + "stopIndex" : 142, + "fragment" : "sum(distinct a.four + b.four)" + } ] +} + + +-- !query +select + (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1))) +from tenk1 o +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`o`.`unique1`", + "proposal" : "`i`.`unique1`, `i`.`unique2`, `i`.`hundred`, `i`.`even`, `i`.`four`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 79, + "fragment" : "o.unique1" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part2.sql.out new file mode 100644 index 0000000000000..f71a78abe5319 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part2.sql.out @@ -0,0 +1,371 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1) +-- !query analysis +CreateViewCommand `int4_tbl`, select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1), false, false, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias int4_tbl + +- LocalRelation [f1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4) +-- !query analysis +CreateViewCommand `bitwise_test`, SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4), false, true, LocalTempView, true + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0 +-- !query analysis +Aggregate [bit_and(b1#x) AS n1#x, bit_or(b2#x) AS n2#x] ++- Filter (1 = 0) + +- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null +-- !query analysis +Aggregate [bit_and(b4#xL) AS n1#xL, bit_or(b4#xL) AS n2#xL] ++- Filter isnull(b4#xL) + +- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT + BIT_AND(cast(b1 as tinyint)) AS a1, + BIT_AND(cast(b2 as smallint)) AS b1, + BIT_AND(b3) AS c1, + BIT_AND(b4) AS d1, + BIT_OR(cast(b1 as tinyint)) AS e7, + BIT_OR(cast(b2 as smallint)) AS f7, + BIT_OR(b3) AS g7, + BIT_OR(b4) AS h3 +FROM bitwise_test +-- !query analysis +Aggregate [bit_and(cast(b1#x as tinyint)) AS a1#x, bit_and(cast(b2#x as smallint)) AS b1#x, bit_and(b3#x) AS c1#x, bit_and(b4#xL) AS d1#xL, bit_or(cast(b1#x as tinyint)) AS e7#x, bit_or(cast(b2#x as smallint)) AS f7#x, bit_or(b3#x) AS g7#x, bit_or(b4#xL) AS h3#xL] ++- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT b1 , bit_and(b2), bit_or(b4) FROM bitwise_test GROUP BY b1 +-- !query analysis +Aggregate [b1#x], [b1#x, bit_and(b2#x) AS bit_and(b2)#x, bit_or(b4#xL) AS bit_or(b4)#xL] ++- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT b1, bit_and(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7 +-- !query analysis +Filter (bit_and(b2)#x < 7) ++- Aggregate [b1#x], [b1#x, bit_and(b2#x) AS bit_and(b2)#x] + +- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT b1, b2, bit_and(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test +-- !query analysis +Project [b1#x, b2#x, bit_and(b2) OVER (PARTITION BY b1 ORDER BY b2 ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [b1#x, b2#x, bit_and(b2) OVER (PARTITION BY b1 ORDER BY b2 ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, bit_and(b2) OVER (PARTITION BY b1 ORDER BY b2 ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [bit_and(b2#x) windowspecdefinition(b1#x, b2#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS bit_and(b2) OVER (PARTITION BY b1 ORDER BY b2 ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [b1#x], [b2#x ASC NULLS FIRST] + +- Project [b1#x, b2#x] + +- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT b1, b2, bit_or(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test +-- !query analysis +Project [b1#x, b2#x, bit_or(b2) OVER (PARTITION BY b1 ORDER BY b2 ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [b1#x, b2#x, bit_or(b2) OVER (PARTITION BY b1 ORDER BY b2 ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, bit_or(b2) OVER (PARTITION BY b1 ORDER BY b2 ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [bit_or(b2#x) windowspecdefinition(b1#x, b2#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS bit_or(b2) OVER (PARTITION BY b1 ORDER BY b2 ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [b1#x], [b2#x ASC NULLS FIRST] + +- Project [b1#x, b2#x] + +- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT + (NULL AND NULL) IS NULL AS `t`, + (TRUE AND NULL) IS NULL AS `t`, + (FALSE AND NULL) IS NULL AS `t`, + (NULL AND TRUE) IS NULL AS `t`, + (NULL AND FALSE) IS NULL AS `t`, + (TRUE AND TRUE) AS `t`, + NOT (TRUE AND FALSE) AS `t`, + NOT (FALSE AND TRUE) AS `t`, + NOT (FALSE AND FALSE) AS `t` +-- !query analysis +Project [isnull((null AND null)) AS t#x, isnull((true AND cast(null as boolean))) AS t#x, isnull((false AND cast(null as boolean))) AS t#x, isnull((cast(null as boolean) AND true)) AS t#x, isnull((cast(null as boolean) AND false)) AS t#x, (true AND true) AS t#x, NOT (true AND false) AS t#x, NOT (false AND true) AS t#x, NOT (false AND false) AS t#x] ++- OneRowRelation + + +-- !query +SELECT + (NULL OR NULL) IS NULL AS `t`, + (TRUE OR NULL) IS NULL AS `t`, + (FALSE OR NULL) IS NULL AS `t`, + (NULL OR TRUE) IS NULL AS `t`, + (NULL OR FALSE) IS NULL AS `t`, + (TRUE OR TRUE) AS `t`, + (TRUE OR FALSE) AS `t`, + (FALSE OR TRUE) AS `t`, + NOT (FALSE OR FALSE) AS `t` +-- !query analysis +Project [isnull((null OR null)) AS t#x, isnull((true OR cast(null as boolean))) AS t#x, isnull((false OR cast(null as boolean))) AS t#x, isnull((cast(null as boolean) OR true)) AS t#x, isnull((cast(null as boolean) OR false)) AS t#x, (true OR true) AS t#x, (true OR false) AS t#x, (false OR true) AS t#x, NOT (false OR false) AS t#x] ++- OneRowRelation + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4) +-- !query analysis +CreateViewCommand `bool_test`, SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4), false, true, LocalTempView, true + +- Project [b1#x, b2#x, b3#x, b4#x] + +- SubqueryAlias bool_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#x] + + +-- !query +SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0 +-- !query analysis +Aggregate [bool_and(b1#x) AS n1#x, bool_or(b3#x) AS n2#x] ++- Filter (1 = 0) + +- SubqueryAlias bool_test + +- View (`bool_test`, [b1#x,b2#x,b3#x,b4#x]) + +- Project [cast(b1#x as boolean) AS b1#x, cast(b2#x as boolean) AS b2#x, cast(b3#x as boolean) AS b3#x, cast(b4#x as void) AS b4#x] + +- Project [b1#x, b2#x, b3#x, b4#x] + +- SubqueryAlias bool_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#x] + + +-- !query +SELECT + BOOL_AND(b1) AS f1, + BOOL_AND(b2) AS t2, + BOOL_AND(b3) AS f3, + BOOL_AND(b4) AS n4, + BOOL_AND(NOT b2) AS f5, + BOOL_AND(NOT b3) AS t6 +FROM bool_test +-- !query analysis +Aggregate [bool_and(b1#x) AS f1#x, bool_and(b2#x) AS t2#x, bool_and(b3#x) AS f3#x, bool_and(cast(b4#x as boolean)) AS n4#x, bool_and(NOT b2#x) AS f5#x, bool_and(NOT b3#x) AS t6#x] ++- SubqueryAlias bool_test + +- View (`bool_test`, [b1#x,b2#x,b3#x,b4#x]) + +- Project [cast(b1#x as boolean) AS b1#x, cast(b2#x as boolean) AS b2#x, cast(b3#x as boolean) AS b3#x, cast(b4#x as void) AS b4#x] + +- Project [b1#x, b2#x, b3#x, b4#x] + +- SubqueryAlias bool_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#x] + + +-- !query +SELECT + EVERY(b1) AS f1, + EVERY(b2) AS t2, + EVERY(b3) AS f3, + EVERY(b4) AS n4, + EVERY(NOT b2) AS f5, + EVERY(NOT b3) AS t6 +FROM bool_test +-- !query analysis +Aggregate [every(b1#x) AS f1#x, every(b2#x) AS t2#x, every(b3#x) AS f3#x, every(cast(b4#x as boolean)) AS n4#x, every(NOT b2#x) AS f5#x, every(NOT b3#x) AS t6#x] ++- SubqueryAlias bool_test + +- View (`bool_test`, [b1#x,b2#x,b3#x,b4#x]) + +- Project [cast(b1#x as boolean) AS b1#x, cast(b2#x as boolean) AS b2#x, cast(b3#x as boolean) AS b3#x, cast(b4#x as void) AS b4#x] + +- Project [b1#x, b2#x, b3#x, b4#x] + +- SubqueryAlias bool_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#x] + + +-- !query +SELECT + BOOL_OR(b1) AS t1, + BOOL_OR(b2) AS t2, + BOOL_OR(b3) AS f3, + BOOL_OR(b4) AS n4, + BOOL_OR(NOT b2) AS f5, + BOOL_OR(NOT b3) AS t6 +FROM bool_test +-- !query analysis +Aggregate [bool_or(b1#x) AS t1#x, bool_or(b2#x) AS t2#x, bool_or(b3#x) AS f3#x, bool_or(cast(b4#x as boolean)) AS n4#x, bool_or(NOT b2#x) AS f5#x, bool_or(NOT b3#x) AS t6#x] ++- SubqueryAlias bool_test + +- View (`bool_test`, [b1#x,b2#x,b3#x,b4#x]) + +- Project [cast(b1#x as boolean) AS b1#x, cast(b2#x as boolean) AS b2#x, cast(b3#x as boolean) AS b3#x, cast(b4#x as void) AS b4#x] + +- Project [b1#x, b2#x, b3#x, b4#x] + +- SubqueryAlias bool_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#x] + + +-- !query +select min(unique1) from tenk1 +-- !query analysis +Aggregate [min(unique1#x) AS min(unique1)#x] ++- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select max(unique1) from tenk1 +-- !query analysis +Aggregate [max(unique1#x) AS max(unique1)#x] ++- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select max(unique1) from tenk1 where unique1 < 42 +-- !query analysis +Aggregate [max(unique1#x) AS max(unique1)#x] ++- Filter (unique1#x < 42) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select max(unique1) from tenk1 where unique1 > 42 +-- !query analysis +Aggregate [max(unique1#x) AS max(unique1)#x] ++- Filter (unique1#x > 42) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select max(unique1) from tenk1 where unique1 > 42000 +-- !query analysis +Aggregate [max(unique1#x) AS max(unique1)#x] ++- Filter (unique1#x > 42000) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select max(tenthous) from tenk1 where thousand = 33 +-- !query analysis +Aggregate [max(tenthous#x) AS max(tenthous)#x] ++- Filter (thousand#x = 33) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select min(tenthous) from tenk1 where thousand = 33 +-- !query analysis +Aggregate [min(tenthous#x) AS min(tenthous)#x] ++- Filter (thousand#x = 33) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select distinct max(unique2) from tenk1 +-- !query analysis +Distinct ++- Aggregate [max(unique2#x) AS max(unique2)#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select max(unique2) from tenk1 order by 1 +-- !query analysis +Sort [max(unique2)#x ASC NULLS FIRST], true ++- Aggregate [max(unique2#x) AS max(unique2)#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select max(unique2) from tenk1 order by max(unique2) +-- !query analysis +Sort [max(unique2)#x ASC NULLS FIRST], true ++- Aggregate [max(unique2#x) AS max(unique2)#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select max(unique2) from tenk1 order by max(unique2)+1 +-- !query analysis +Sort [(max(unique2)#x + 1) ASC NULLS FIRST], true ++- Aggregate [max(unique2#x) AS max(unique2)#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select t1.max_unique2, g from (select max(unique2) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc +-- !query analysis +Sort [g#x DESC NULLS LAST], true ++- Project [max_unique2#x, g#x] + +- Generate explode(array(1, 2, 3)), false, t2, [g#x] + +- SubqueryAlias t1 + +- Aggregate [max(unique2#x) AS max_unique2#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select max(100) from tenk1 +-- !query analysis +Aggregate [max(100) AS max(100)#x] ++- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part3.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part3.sql.out new file mode 100644 index 0000000000000..2866c7d26c97f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part3.sql.out @@ -0,0 +1,65 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select max(min(unique1)) from tenk1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NESTED_AGGREGATE_FUNCTION", + "sqlState" : "42607", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 23, + "fragment" : "min(unique1)" + } ] +} + + +-- !query +select min(unique1) filter (where unique1 > 100) from tenk1 +-- !query analysis +Aggregate [min(unique1#x) FILTER (WHERE (unique1#x > 100)) AS min(unique1) FILTER (WHERE (unique1 > 100))#x] ++- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select sum(1/ten) filter (where ten > 0) from tenk1 +-- !query analysis +Aggregate [sum((cast(1 as double) / cast(ten#x as double))) FILTER (WHERE (ten#x > 0)) AS sum((1 / ten)) FILTER (WHERE (ten > 0))#x] ++- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select ten, sum(distinct four) filter (where four > 10) from onek a +group by ten +having exists (select 1 from onek b where sum(distinct a.four) = b.four) +-- !query analysis +Project [ten#x, sum(DISTINCT four) FILTER (WHERE (four > 10))#xL] ++- Filter exists#x [sum(distinct four#x)#xL] + : +- Project [1 AS 1#x] + : +- Filter (outer(sum(distinct four#x)#xL) = cast(four#x as bigint)) + : +- SubqueryAlias b + : +- SubqueryAlias spark_catalog.default.onek + : +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- Aggregate [ten#x], [ten#x, sum(distinct four#x) FILTER (WHERE (four#x > 10)) AS sum(DISTINCT four) FILTER (WHERE (four > 10))#xL, sum(distinct four#x) AS sum(distinct four#x)#xL] + +- SubqueryAlias a + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select (select count(*) + from (values (1)) t0(inner_c)) +from (values (2),(3)) t1(outer_c) +-- !query analysis +Project [scalar-subquery#x [] AS scalarsubquery()#xL] +: +- Aggregate [count(1) AS count(1)#xL] +: +- SubqueryAlias t0 +: +- Project [col1#x AS inner_c#x] +: +- LocalRelation [col1#x] ++- SubqueryAlias t1 + +- Project [col1#x AS outer_c#x] + +- LocalRelation [col1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part4.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part4.sql.out new file mode 100644 index 0000000000000..6b33408ba1a63 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part4.sql.out @@ -0,0 +1,31 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select percentile_cont(0.5) within group (order by b) from aggtest +-- !query analysis +Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +select percentile_cont(0.5) within group (order by b), sum(b) from aggtest +-- !query analysis +Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x, sum(b#x) AS sum(b)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +select percentile_cont(0.5) within group (order by thousand) from tenk1 +-- !query analysis +Aggregate [percentile_cont(thousand#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x] ++- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select percentile_disc(0.5) within group (order by thousand) from tenk1 +-- !query analysis +Aggregate [percentile_disc(thousand#x, cast(0.5 as double), false, 0, 0) AS percentile_disc(0.5) WITHIN GROUP (ORDER BY v)#x] ++- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/boolean.sql.out new file mode 100644 index 0000000000000..0ec0b55bb27e9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/boolean.sql.out @@ -0,0 +1,765 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT 1 AS one +-- !query analysis +Project [1 AS one#x] ++- OneRowRelation + + +-- !query +SELECT true AS true +-- !query analysis +Project [true AS true#x] ++- OneRowRelation + + +-- !query +SELECT false AS `false` +-- !query analysis +Project [false AS false#x] ++- OneRowRelation + + +-- !query +SELECT boolean('t') AS true +-- !query analysis +Project [cast(t as boolean) AS true#x] ++- OneRowRelation + + +-- !query +SELECT boolean(' f ') AS `false` +-- !query analysis +Project [cast( f as boolean) AS false#x] ++- OneRowRelation + + +-- !query +SELECT boolean('true') AS true +-- !query analysis +Project [cast(true as boolean) AS true#x] ++- OneRowRelation + + +-- !query +SELECT boolean('test') AS error +-- !query analysis +Project [cast(test as boolean) AS error#x] ++- OneRowRelation + + +-- !query +SELECT boolean('false') AS `false` +-- !query analysis +Project [cast(false as boolean) AS false#x] ++- OneRowRelation + + +-- !query +SELECT boolean('foo') AS error +-- !query analysis +Project [cast(foo as boolean) AS error#x] ++- OneRowRelation + + +-- !query +SELECT boolean('y') AS true +-- !query analysis +Project [cast(y as boolean) AS true#x] ++- OneRowRelation + + +-- !query +SELECT boolean('yes') AS true +-- !query analysis +Project [cast(yes as boolean) AS true#x] ++- OneRowRelation + + +-- !query +SELECT boolean('yeah') AS error +-- !query analysis +Project [cast(yeah as boolean) AS error#x] ++- OneRowRelation + + +-- !query +SELECT boolean('n') AS `false` +-- !query analysis +Project [cast(n as boolean) AS false#x] ++- OneRowRelation + + +-- !query +SELECT boolean('no') AS `false` +-- !query analysis +Project [cast(no as boolean) AS false#x] ++- OneRowRelation + + +-- !query +SELECT boolean('nay') AS error +-- !query analysis +Project [cast(nay as boolean) AS error#x] ++- OneRowRelation + + +-- !query +SELECT boolean('on') AS true +-- !query analysis +Project [cast(on as boolean) AS true#x] ++- OneRowRelation + + +-- !query +SELECT boolean('off') AS `false` +-- !query analysis +Project [cast(off as boolean) AS false#x] ++- OneRowRelation + + +-- !query +SELECT boolean('of') AS `false` +-- !query analysis +Project [cast(of as boolean) AS false#x] ++- OneRowRelation + + +-- !query +SELECT boolean('o') AS error +-- !query analysis +Project [cast(o as boolean) AS error#x] ++- OneRowRelation + + +-- !query +SELECT boolean('on_') AS error +-- !query analysis +Project [cast(on_ as boolean) AS error#x] ++- OneRowRelation + + +-- !query +SELECT boolean('off_') AS error +-- !query analysis +Project [cast(off_ as boolean) AS error#x] ++- OneRowRelation + + +-- !query +SELECT boolean('1') AS true +-- !query analysis +Project [cast(1 as boolean) AS true#x] ++- OneRowRelation + + +-- !query +SELECT boolean('11') AS error +-- !query analysis +Project [cast(11 as boolean) AS error#x] ++- OneRowRelation + + +-- !query +SELECT boolean('0') AS `false` +-- !query analysis +Project [cast(0 as boolean) AS false#x] ++- OneRowRelation + + +-- !query +SELECT boolean('000') AS error +-- !query analysis +Project [cast(000 as boolean) AS error#x] ++- OneRowRelation + + +-- !query +SELECT boolean('') AS error +-- !query analysis +Project [cast( as boolean) AS error#x] ++- OneRowRelation + + +-- !query +SELECT boolean('t') or boolean('f') AS true +-- !query analysis +Project [(cast(t as boolean) OR cast(f as boolean)) AS true#x] ++- OneRowRelation + + +-- !query +SELECT boolean('t') and boolean('f') AS `false` +-- !query analysis +Project [(cast(t as boolean) AND cast(f as boolean)) AS false#x] ++- OneRowRelation + + +-- !query +SELECT not boolean('f') AS true +-- !query analysis +Project [NOT cast(f as boolean) AS true#x] ++- OneRowRelation + + +-- !query +SELECT boolean('t') = boolean('f') AS `false` +-- !query analysis +Project [(cast(t as boolean) = cast(f as boolean)) AS false#x] ++- OneRowRelation + + +-- !query +SELECT boolean('t') <> boolean('f') AS true +-- !query analysis +Project [NOT (cast(t as boolean) = cast(f as boolean)) AS true#x] ++- OneRowRelation + + +-- !query +SELECT boolean('t') > boolean('f') AS true +-- !query analysis +Project [(cast(t as boolean) > cast(f as boolean)) AS true#x] ++- OneRowRelation + + +-- !query +SELECT boolean('t') >= boolean('f') AS true +-- !query analysis +Project [(cast(t as boolean) >= cast(f as boolean)) AS true#x] ++- OneRowRelation + + +-- !query +SELECT boolean('f') < boolean('t') AS true +-- !query analysis +Project [(cast(f as boolean) < cast(t as boolean)) AS true#x] ++- OneRowRelation + + +-- !query +SELECT boolean('f') <= boolean('t') AS true +-- !query analysis +Project [(cast(f as boolean) <= cast(t as boolean)) AS true#x] ++- OneRowRelation + + +-- !query +SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS `false` +-- !query analysis +Project [cast(cast(TrUe as string) as boolean) AS true#x, cast(cast(fAlse as string) as boolean) AS false#x] ++- OneRowRelation + + +-- !query +SELECT boolean(string(' true ')) AS true, + boolean(string(' FALSE')) AS `false` +-- !query analysis +Project [cast(cast( true as string) as boolean) AS true#x, cast(cast( FALSE as string) as boolean) AS false#x] ++- OneRowRelation + + +-- !query +SELECT string(boolean(true)) AS true, string(boolean(false)) AS `false` +-- !query analysis +Project [cast(cast(true as boolean) as string) AS true#x, cast(cast(false as boolean) as string) AS false#x] ++- OneRowRelation + + +-- !query +SELECT boolean(string(' tru e ')) AS invalid +-- !query analysis +Project [cast(cast( tru e as string) as boolean) AS invalid#x] ++- OneRowRelation + + +-- !query +SELECT boolean(string('')) AS invalid +-- !query analysis +Project [cast(cast( as string) as boolean) AS invalid#x] ++- OneRowRelation + + +-- !query +CREATE TABLE BOOLTBL1 (f1 boolean) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`BOOLTBL1`, false + + +-- !query +INSERT INTO BOOLTBL1 VALUES (cast('t' as boolean)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl1], Append, `spark_catalog`.`default`.`booltbl1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl1), [f1] ++- Project [cast(col1#x as boolean) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO BOOLTBL1 VALUES (cast('True' as boolean)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl1], Append, `spark_catalog`.`default`.`booltbl1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl1), [f1] ++- Project [cast(col1#x as boolean) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO BOOLTBL1 VALUES (cast('true' as boolean)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl1], Append, `spark_catalog`.`default`.`booltbl1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl1), [f1] ++- Project [cast(col1#x as boolean) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT '' AS t_3, BOOLTBL1.* FROM BOOLTBL1 +-- !query analysis +Project [ AS t_3#x, f1#x] ++- SubqueryAlias spark_catalog.default.booltbl1 + +- Relation spark_catalog.default.booltbl1[f1#x] parquet + + +-- !query +SELECT '' AS t_3, BOOLTBL1.* + FROM BOOLTBL1 + WHERE f1 = boolean('true') +-- !query analysis +Project [ AS t_3#x, f1#x] ++- Filter (f1#x = cast(true as boolean)) + +- SubqueryAlias spark_catalog.default.booltbl1 + +- Relation spark_catalog.default.booltbl1[f1#x] parquet + + +-- !query +SELECT '' AS t_3, BOOLTBL1.* + FROM BOOLTBL1 + WHERE f1 <> boolean('false') +-- !query analysis +Project [ AS t_3#x, f1#x] ++- Filter NOT (f1#x = cast(false as boolean)) + +- SubqueryAlias spark_catalog.default.booltbl1 + +- Relation spark_catalog.default.booltbl1[f1#x] parquet + + +-- !query +SELECT '' AS zero, BOOLTBL1.* + FROM BOOLTBL1 + WHERE booleq(boolean('false'), f1) +-- !query analysis +Project [ AS zero#x, f1#x] ++- Filter if ((isnull(cast(false as boolean)) OR isnull(f1#x))) null else booleq(knownnotnull(cast(false as boolean)), knownnotnull(f1#x)) + +- SubqueryAlias spark_catalog.default.booltbl1 + +- Relation spark_catalog.default.booltbl1[f1#x] parquet + + +-- !query +INSERT INTO BOOLTBL1 VALUES (boolean('f')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl1], Append, `spark_catalog`.`default`.`booltbl1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl1), [f1] ++- Project [cast(col1#x as boolean) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT '' AS f_1, BOOLTBL1.* + FROM BOOLTBL1 + WHERE f1 = boolean('false') +-- !query analysis +Project [ AS f_1#x, f1#x] ++- Filter (f1#x = cast(false as boolean)) + +- SubqueryAlias spark_catalog.default.booltbl1 + +- Relation spark_catalog.default.booltbl1[f1#x] parquet + + +-- !query +CREATE TABLE BOOLTBL2 (f1 boolean) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`BOOLTBL2`, false + + +-- !query +INSERT INTO BOOLTBL2 VALUES (boolean('f')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl2], Append, `spark_catalog`.`default`.`booltbl2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl2), [f1] ++- Project [cast(col1#x as boolean) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO BOOLTBL2 VALUES (boolean('false')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl2], Append, `spark_catalog`.`default`.`booltbl2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl2), [f1] ++- Project [cast(col1#x as boolean) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO BOOLTBL2 VALUES (boolean('False')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl2], Append, `spark_catalog`.`default`.`booltbl2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl2), [f1] ++- Project [cast(col1#x as boolean) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO BOOLTBL2 VALUES (boolean('FALSE')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl2], Append, `spark_catalog`.`default`.`booltbl2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl2), [f1] ++- Project [cast(col1#x as boolean) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO BOOLTBL2 + VALUES (boolean('XXX')) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2331", + "messageParameters" : { + "msg" : "[CAST_INVALID_INPUT] The value 'XXX' of the type \"STRING\" cannot be cast to \"BOOLEAN\" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.\n== SQL(line 2, position 12) ==\n VALUES (boolean('XXX'))\n ^^^^^^^^^^^^^^\n", + "sqlExpr" : "CAST('XXX' AS BOOLEAN)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 47, + "fragment" : "VALUES (boolean('XXX'))" + } ] +} + + +-- !query +SELECT '' AS f_4, BOOLTBL2.* FROM BOOLTBL2 +-- !query analysis +Project [ AS f_4#x, f1#x] ++- SubqueryAlias spark_catalog.default.booltbl2 + +- Relation spark_catalog.default.booltbl2[f1#x] parquet + + +-- !query +SELECT '' AS tf_12, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE BOOLTBL2.f1 <> BOOLTBL1.f1 +-- !query analysis +Project [ AS tf_12#x, f1#x, f1#x] ++- Filter NOT (f1#x = f1#x) + +- Join Inner + :- SubqueryAlias spark_catalog.default.booltbl1 + : +- Relation spark_catalog.default.booltbl1[f1#x] parquet + +- SubqueryAlias spark_catalog.default.booltbl2 + +- Relation spark_catalog.default.booltbl2[f1#x] parquet + + +-- !query +SELECT '' AS tf_12, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE boolne(BOOLTBL2.f1,BOOLTBL1.f1) +-- !query analysis +Project [ AS tf_12#x, f1#x, f1#x] ++- Filter if ((isnull(f1#x) OR isnull(f1#x))) null else boolne(knownnotnull(f1#x), knownnotnull(f1#x)) + +- Join Inner + :- SubqueryAlias spark_catalog.default.booltbl1 + : +- Relation spark_catalog.default.booltbl1[f1#x] parquet + +- SubqueryAlias spark_catalog.default.booltbl2 + +- Relation spark_catalog.default.booltbl2[f1#x] parquet + + +-- !query +SELECT '' AS ff_4, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE BOOLTBL2.f1 = BOOLTBL1.f1 and BOOLTBL1.f1 = boolean('false') +-- !query analysis +Project [ AS ff_4#x, f1#x, f1#x] ++- Filter ((f1#x = f1#x) AND (f1#x = cast(false as boolean))) + +- Join Inner + :- SubqueryAlias spark_catalog.default.booltbl1 + : +- Relation spark_catalog.default.booltbl1[f1#x] parquet + +- SubqueryAlias spark_catalog.default.booltbl2 + +- Relation spark_catalog.default.booltbl2[f1#x] parquet + + +-- !query +SELECT '' AS tf_12_ff_4, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE BOOLTBL2.f1 = BOOLTBL1.f1 or BOOLTBL1.f1 = boolean('true') + ORDER BY BOOLTBL1.f1, BOOLTBL2.f1 +-- !query analysis +Sort [f1#x ASC NULLS FIRST, f1#x ASC NULLS FIRST], true ++- Project [ AS tf_12_ff_4#x, f1#x, f1#x] + +- Filter ((f1#x = f1#x) OR (f1#x = cast(true as boolean))) + +- Join Inner + :- SubqueryAlias spark_catalog.default.booltbl1 + : +- Relation spark_catalog.default.booltbl1[f1#x] parquet + +- SubqueryAlias spark_catalog.default.booltbl2 + +- Relation spark_catalog.default.booltbl2[f1#x] parquet + + +-- !query +SELECT '' AS True, f1 + FROM BOOLTBL1 + WHERE f1 IS TRUE +-- !query analysis +Project [ AS True#x, f1#x] ++- Filter (f1#x <=> true) + +- SubqueryAlias spark_catalog.default.booltbl1 + +- Relation spark_catalog.default.booltbl1[f1#x] parquet + + +-- !query +SELECT '' AS `Not False`, f1 + FROM BOOLTBL1 + WHERE f1 IS NOT FALSE +-- !query analysis +Project [ AS Not False#x, f1#x] ++- Filter NOT (f1#x <=> false) + +- SubqueryAlias spark_catalog.default.booltbl1 + +- Relation spark_catalog.default.booltbl1[f1#x] parquet + + +-- !query +SELECT '' AS `False`, f1 + FROM BOOLTBL1 + WHERE f1 IS FALSE +-- !query analysis +Project [ AS False#x, f1#x] ++- Filter (f1#x <=> false) + +- SubqueryAlias spark_catalog.default.booltbl1 + +- Relation spark_catalog.default.booltbl1[f1#x] parquet + + +-- !query +SELECT '' AS `Not True`, f1 + FROM BOOLTBL1 + WHERE f1 IS NOT TRUE +-- !query analysis +Project [ AS Not True#x, f1#x] ++- Filter NOT (f1#x <=> true) + +- SubqueryAlias spark_catalog.default.booltbl1 + +- Relation spark_catalog.default.booltbl1[f1#x] parquet + + +-- !query +SELECT '' AS `True`, f1 + FROM BOOLTBL2 + WHERE f1 IS TRUE +-- !query analysis +Project [ AS True#x, f1#x] ++- Filter (f1#x <=> true) + +- SubqueryAlias spark_catalog.default.booltbl2 + +- Relation spark_catalog.default.booltbl2[f1#x] parquet + + +-- !query +SELECT '' AS `Not False`, f1 + FROM BOOLTBL2 + WHERE f1 IS NOT FALSE +-- !query analysis +Project [ AS Not False#x, f1#x] ++- Filter NOT (f1#x <=> false) + +- SubqueryAlias spark_catalog.default.booltbl2 + +- Relation spark_catalog.default.booltbl2[f1#x] parquet + + +-- !query +SELECT '' AS `False`, f1 + FROM BOOLTBL2 + WHERE f1 IS FALSE +-- !query analysis +Project [ AS False#x, f1#x] ++- Filter (f1#x <=> false) + +- SubqueryAlias spark_catalog.default.booltbl2 + +- Relation spark_catalog.default.booltbl2[f1#x] parquet + + +-- !query +SELECT '' AS `Not True`, f1 + FROM BOOLTBL2 + WHERE f1 IS NOT TRUE +-- !query analysis +Project [ AS Not True#x, f1#x] ++- Filter NOT (f1#x <=> true) + +- SubqueryAlias spark_catalog.default.booltbl2 + +- Relation spark_catalog.default.booltbl2[f1#x] parquet + + +-- !query +CREATE TABLE BOOLTBL3 (d string, b boolean, o int) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`BOOLTBL3`, false + + +-- !query +INSERT INTO BOOLTBL3 VALUES ('true', true, 1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl3], Append, `spark_catalog`.`default`.`booltbl3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl3), [d, b, o] ++- Project [cast(col1#x as string) AS d#x, cast(col2#x as boolean) AS b#x, cast(col3#x as int) AS o#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO BOOLTBL3 VALUES ('false', false, 2) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl3], Append, `spark_catalog`.`default`.`booltbl3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl3), [d, b, o] ++- Project [cast(col1#x as string) AS d#x, cast(col2#x as boolean) AS b#x, cast(col3#x as int) AS o#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO BOOLTBL3 VALUES ('null', null, 3) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl3], Append, `spark_catalog`.`default`.`booltbl3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl3), [d, b, o] ++- Project [cast(col1#x as string) AS d#x, cast(col2#x as boolean) AS b#x, cast(col3#x as int) AS o#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT + d, + b IS TRUE AS istrue, + b IS NOT TRUE AS isnottrue, + b IS FALSE AS isfalse, + b IS NOT FALSE AS isnotfalse, + b IS UNKNOWN AS isunknown, + b IS NOT UNKNOWN AS isnotunknown +FROM booltbl3 ORDER BY o +-- !query analysis +Project [d#x, istrue#x, isnottrue#x, isfalse#x, isnotfalse#x, isunknown#x, isnotunknown#x] ++- Sort [o#x ASC NULLS FIRST], true + +- Project [d#x, (b#x <=> true) AS istrue#x, NOT (b#x <=> true) AS isnottrue#x, (b#x <=> false) AS isfalse#x, NOT (b#x <=> false) AS isnotfalse#x, isnull(b#x) AS isunknown#x, isnotnull(b#x) AS isnotunknown#x, o#x] + +- SubqueryAlias spark_catalog.default.booltbl3 + +- Relation spark_catalog.default.booltbl3[d#x,b#x,o#x] parquet + + +-- !query +CREATE TABLE booltbl4(isfalse boolean, istrue boolean, isnul boolean) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`booltbl4`, false + + +-- !query +INSERT INTO booltbl4 VALUES (false, true, null) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl4, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl4], Append, `spark_catalog`.`default`.`booltbl4`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl4), [isfalse, istrue, isnul] ++- Project [cast(col1#x as boolean) AS isfalse#x, cast(col2#x as boolean) AS istrue#x, cast(col3#x as boolean) AS isnul#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT istrue AND isnul AND istrue FROM booltbl4 +-- !query analysis +Project [((istrue#x AND isnul#x) AND istrue#x) AS ((istrue AND isnul) AND istrue)#x] ++- SubqueryAlias spark_catalog.default.booltbl4 + +- Relation spark_catalog.default.booltbl4[isfalse#x,istrue#x,isnul#x] parquet + + +-- !query +SELECT istrue AND istrue AND isnul FROM booltbl4 +-- !query analysis +Project [((istrue#x AND istrue#x) AND isnul#x) AS ((istrue AND istrue) AND isnul)#x] ++- SubqueryAlias spark_catalog.default.booltbl4 + +- Relation spark_catalog.default.booltbl4[isfalse#x,istrue#x,isnul#x] parquet + + +-- !query +SELECT isnul AND istrue AND istrue FROM booltbl4 +-- !query analysis +Project [((isnul#x AND istrue#x) AND istrue#x) AS ((isnul AND istrue) AND istrue)#x] ++- SubqueryAlias spark_catalog.default.booltbl4 + +- Relation spark_catalog.default.booltbl4[isfalse#x,istrue#x,isnul#x] parquet + + +-- !query +SELECT isfalse AND isnul AND istrue FROM booltbl4 +-- !query analysis +Project [((isfalse#x AND isnul#x) AND istrue#x) AS ((isfalse AND isnul) AND istrue)#x] ++- SubqueryAlias spark_catalog.default.booltbl4 + +- Relation spark_catalog.default.booltbl4[isfalse#x,istrue#x,isnul#x] parquet + + +-- !query +SELECT istrue AND isfalse AND isnul FROM booltbl4 +-- !query analysis +Project [((istrue#x AND isfalse#x) AND isnul#x) AS ((istrue AND isfalse) AND isnul)#x] ++- SubqueryAlias spark_catalog.default.booltbl4 + +- Relation spark_catalog.default.booltbl4[isfalse#x,istrue#x,isnul#x] parquet + + +-- !query +SELECT isnul AND istrue AND isfalse FROM booltbl4 +-- !query analysis +Project [((isnul#x AND istrue#x) AND isfalse#x) AS ((isnul AND istrue) AND isfalse)#x] ++- SubqueryAlias spark_catalog.default.booltbl4 + +- Relation spark_catalog.default.booltbl4[isfalse#x,istrue#x,isnul#x] parquet + + +-- !query +SELECT isfalse OR isnul OR isfalse FROM booltbl4 +-- !query analysis +Project [((isfalse#x OR isnul#x) OR isfalse#x) AS ((isfalse OR isnul) OR isfalse)#x] ++- SubqueryAlias spark_catalog.default.booltbl4 + +- Relation spark_catalog.default.booltbl4[isfalse#x,istrue#x,isnul#x] parquet + + +-- !query +SELECT isfalse OR isfalse OR isnul FROM booltbl4 +-- !query analysis +Project [((isfalse#x OR isfalse#x) OR isnul#x) AS ((isfalse OR isfalse) OR isnul)#x] ++- SubqueryAlias spark_catalog.default.booltbl4 + +- Relation spark_catalog.default.booltbl4[isfalse#x,istrue#x,isnul#x] parquet + + +-- !query +SELECT isnul OR isfalse OR isfalse FROM booltbl4 +-- !query analysis +Project [((isnul#x OR isfalse#x) OR isfalse#x) AS ((isnul OR isfalse) OR isfalse)#x] ++- SubqueryAlias spark_catalog.default.booltbl4 + +- Relation spark_catalog.default.booltbl4[isfalse#x,istrue#x,isnul#x] parquet + + +-- !query +SELECT isfalse OR isnul OR istrue FROM booltbl4 +-- !query analysis +Project [((isfalse#x OR isnul#x) OR istrue#x) AS ((isfalse OR isnul) OR istrue)#x] ++- SubqueryAlias spark_catalog.default.booltbl4 + +- Relation spark_catalog.default.booltbl4[isfalse#x,istrue#x,isnul#x] parquet + + +-- !query +SELECT istrue OR isfalse OR isnul FROM booltbl4 +-- !query analysis +Project [((istrue#x OR isfalse#x) OR isnul#x) AS ((istrue OR isfalse) OR isnul)#x] ++- SubqueryAlias spark_catalog.default.booltbl4 + +- Relation spark_catalog.default.booltbl4[isfalse#x,istrue#x,isnul#x] parquet + + +-- !query +SELECT isnul OR istrue OR isfalse FROM booltbl4 +-- !query analysis +Project [((isnul#x OR istrue#x) OR isfalse#x) AS ((isnul OR istrue) OR isfalse)#x] ++- SubqueryAlias spark_catalog.default.booltbl4 + +- Relation spark_catalog.default.booltbl4[isfalse#x,istrue#x,isnul#x] parquet + + +-- !query +DROP TABLE BOOLTBL1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`BOOLTBL1`, false, false, false + + +-- !query +DROP TABLE BOOLTBL2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`BOOLTBL2`, false, false, false + + +-- !query +DROP TABLE BOOLTBL3 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`BOOLTBL3`, false, false, false + + +-- !query +DROP TABLE BOOLTBL4 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`BOOLTBL4`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/case.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/case.sql.out new file mode 100644 index 0000000000000..80b658f1d1ee3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/case.sql.out @@ -0,0 +1,348 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE CASE_TBL ( + i integer, + f double +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`CASE_TBL`, false + + +-- !query +CREATE TABLE CASE2_TBL ( + i integer, + j integer +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`CASE2_TBL`, false + + +-- !query +INSERT INTO CASE_TBL VALUES (1, 10.1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case_tbl], Append, `spark_catalog`.`default`.`case_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case_tbl), [i, f] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as double) AS f#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE_TBL VALUES (2, 20.2) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case_tbl], Append, `spark_catalog`.`default`.`case_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case_tbl), [i, f] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as double) AS f#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE_TBL VALUES (3, -30.3) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case_tbl], Append, `spark_catalog`.`default`.`case_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case_tbl), [i, f] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as double) AS f#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE_TBL VALUES (4, NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case_tbl], Append, `spark_catalog`.`default`.`case_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case_tbl), [i, f] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as double) AS f#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE2_TBL VALUES (1, -1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE2_TBL VALUES (2, -2) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE2_TBL VALUES (3, -3) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE2_TBL VALUES (2, -4) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE2_TBL VALUES (1, NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE2_TBL VALUES (NULL, -6) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT '3' AS `One`, + CASE + WHEN 1 < 2 THEN 3 + END AS `Simple WHEN` +-- !query analysis +Project [3 AS One#x, CASE WHEN (1 < 2) THEN 3 END AS Simple WHEN#x] ++- OneRowRelation + + +-- !query +SELECT '' AS `One`, + CASE + WHEN 1 > 2 THEN 3 + END AS `Simple default` +-- !query analysis +Project [ AS One#x, CASE WHEN (1 > 2) THEN 3 END AS Simple default#x] ++- OneRowRelation + + +-- !query +SELECT '3' AS `One`, + CASE + WHEN 1 < 2 THEN 3 + ELSE 4 + END AS `Simple ELSE` +-- !query analysis +Project [3 AS One#x, CASE WHEN (1 < 2) THEN 3 ELSE 4 END AS Simple ELSE#x] ++- OneRowRelation + + +-- !query +SELECT '4' AS `One`, + CASE + WHEN 1 > 2 THEN 3 + ELSE 4 + END AS `ELSE default` +-- !query analysis +Project [4 AS One#x, CASE WHEN (1 > 2) THEN 3 ELSE 4 END AS ELSE default#x] ++- OneRowRelation + + +-- !query +SELECT '6' AS `One`, + CASE + WHEN 1 > 2 THEN 3 + WHEN 4 < 5 THEN 6 + ELSE 7 + END AS `Two WHEN with default` +-- !query analysis +Project [6 AS One#x, CASE WHEN (1 > 2) THEN 3 WHEN (4 < 5) THEN 6 ELSE 7 END AS Two WHEN with default#x] ++- OneRowRelation + + +-- !query +SELECT '7' AS `None`, + CASE WHEN rand() < 0 THEN 1 + END AS `NULL on no matches` +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT CASE WHEN 1=0 THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END +-- !query analysis +Project [CASE WHEN (1 = 0) THEN (cast(1 as double) / cast(0 as double)) WHEN (1 = 1) THEN cast(1 as double) ELSE (cast(2 as double) / cast(0 as double)) END AS CASE WHEN (1 = 0) THEN (1 / 0) WHEN (1 = 1) THEN 1 ELSE (2 / 0) END#x] ++- OneRowRelation + + +-- !query +SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END +-- !query analysis +Project [CASE WHEN (1 = 0) THEN (cast(1 as double) / cast(0 as double)) WHEN (1 = 1) THEN cast(1 as double) ELSE (cast(2 as double) / cast(0 as double)) END AS CASE WHEN (1 = 0) THEN (1 / 0) WHEN (1 = 1) THEN 1 ELSE (2 / 0) END#x] ++- OneRowRelation + + +-- !query +SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl +-- !query analysis +Project [CASE WHEN (i#x > 100) THEN (cast(1 as double) / cast(0 as double)) ELSE cast(0 as double) END AS CASE WHEN (i > 100) THEN (1 / 0) ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.case_tbl + +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + + +-- !query +SELECT CASE 'a' WHEN 'a' THEN 1 ELSE 2 END +-- !query analysis +Project [CASE WHEN (a = a) THEN 1 ELSE 2 END AS CASE WHEN (a = a) THEN 1 ELSE 2 END#x] ++- OneRowRelation + + +-- !query +SELECT '' AS `Five`, + CASE + WHEN i >= 3 THEN i + END AS `>= 3 or Null` + FROM CASE_TBL +-- !query analysis +Project [ AS Five#x, CASE WHEN (i#x >= 3) THEN i#x END AS >= 3 or Null#x] ++- SubqueryAlias spark_catalog.default.case_tbl + +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + + +-- !query +SELECT '' AS `Five`, + CASE WHEN i >= 3 THEN (i + i) + ELSE i + END AS `Simplest Math` + FROM CASE_TBL +-- !query analysis +Project [ AS Five#x, CASE WHEN (i#x >= 3) THEN (i#x + i#x) ELSE i#x END AS Simplest Math#x] ++- SubqueryAlias spark_catalog.default.case_tbl + +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + + +-- !query +SELECT '' AS `Five`, i AS `Value`, + CASE WHEN (i < 0) THEN 'small' + WHEN (i = 0) THEN 'zero' + WHEN (i = 1) THEN 'one' + WHEN (i = 2) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL +-- !query analysis +Project [ AS Five#x, i#x AS Value#x, CASE WHEN (i#x < 0) THEN small WHEN (i#x = 0) THEN zero WHEN (i#x = 1) THEN one WHEN (i#x = 2) THEN two ELSE big END AS Category#x] ++- SubqueryAlias spark_catalog.default.case_tbl + +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + + +-- !query +SELECT '' AS `Five`, + CASE WHEN ((i < 0) or (i < 0)) THEN 'small' + WHEN ((i = 0) or (i = 0)) THEN 'zero' + WHEN ((i = 1) or (i = 1)) THEN 'one' + WHEN ((i = 2) or (i = 2)) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL +-- !query analysis +Project [ AS Five#x, CASE WHEN ((i#x < 0) OR (i#x < 0)) THEN small WHEN ((i#x = 0) OR (i#x = 0)) THEN zero WHEN ((i#x = 1) OR (i#x = 1)) THEN one WHEN ((i#x = 2) OR (i#x = 2)) THEN two ELSE big END AS Category#x] ++- SubqueryAlias spark_catalog.default.case_tbl + +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + + +-- !query +SELECT * FROM CASE_TBL WHERE COALESCE(f,i) = 4 +-- !query analysis +Project [i#x, f#x] ++- Filter (coalesce(f#x, cast(i#x as double)) = cast(4 as double)) + +- SubqueryAlias spark_catalog.default.case_tbl + +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + + +-- !query +SELECT * FROM CASE_TBL WHERE NULLIF(f,i) = 2 +-- !query analysis +Project [i#x, f#x] ++- Filter (nullif(f#x, i#x) = cast(2 as double)) + +- SubqueryAlias spark_catalog.default.case_tbl + +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + + +-- !query +SELECT COALESCE(a.f, b.i, b.j) + FROM CASE_TBL a, CASE2_TBL b +-- !query analysis +Project [coalesce(f#x, cast(i#x as double), cast(j#x as double)) AS coalesce(f, i, j)#x] ++- Join Inner + :- SubqueryAlias a + : +- SubqueryAlias spark_catalog.default.case_tbl + : +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.case2_tbl + +- Relation spark_catalog.default.case2_tbl[i#x,j#x] parquet + + +-- !query +SELECT * + FROM CASE_TBL a, CASE2_TBL b + WHERE COALESCE(a.f, b.i, b.j) = 2 +-- !query analysis +Project [i#x, f#x, i#x, j#x] ++- Filter (coalesce(f#x, cast(i#x as double), cast(j#x as double)) = cast(2 as double)) + +- Join Inner + :- SubqueryAlias a + : +- SubqueryAlias spark_catalog.default.case_tbl + : +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.case2_tbl + +- Relation spark_catalog.default.case2_tbl[i#x,j#x] parquet + + +-- !query +SELECT '' AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, + NULLIF(b.i, 4) AS `NULLIF(b.i,4)` + FROM CASE_TBL a, CASE2_TBL b +-- !query analysis +Project [ AS Five#x, nullif(i#x, i#x) AS NULLIF(a.i,b.i)#x, nullif(i#x, 4) AS NULLIF(b.i,4)#x] ++- Join Inner + :- SubqueryAlias a + : +- SubqueryAlias spark_catalog.default.case_tbl + : +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.case2_tbl + +- Relation spark_catalog.default.case2_tbl[i#x,j#x] parquet + + +-- !query +SELECT '' AS `Two`, * + FROM CASE_TBL a, CASE2_TBL b + WHERE COALESCE(f,b.i) = 2 +-- !query analysis +Project [ AS Two#x, i#x, f#x, i#x, j#x] ++- Filter (coalesce(f#x, cast(i#x as double)) = cast(2 as double)) + +- Join Inner + :- SubqueryAlias a + : +- SubqueryAlias spark_catalog.default.case_tbl + : +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.case2_tbl + +- Relation spark_catalog.default.case2_tbl[i#x,j#x] parquet + + +-- !query +SELECT CASE + (CASE vol('bar') + WHEN 'foo' THEN 'it was foo!' + WHEN vol(null) THEN 'null input' + WHEN 'bar' THEN 'it was bar!' END + ) + WHEN 'it was foo!' THEN 'foo recognized' + WHEN 'it was bar!' THEN 'bar recognized' + ELSE 'unrecognized' END +-- !query analysis +Project [CASE WHEN (CASE WHEN (vol(bar) = foo) THEN it was foo! WHEN (vol(bar) = vol(cast(null as string))) THEN null input WHEN (vol(bar) = bar) THEN it was bar! END = it was foo!) THEN foo recognized WHEN (CASE WHEN (vol(bar) = foo) THEN it was foo! WHEN (vol(bar) = vol(cast(null as string))) THEN null input WHEN (vol(bar) = bar) THEN it was bar! END = it was bar!) THEN bar recognized ELSE unrecognized END AS CASE WHEN (CASE WHEN (vol(bar) = foo) THEN it was foo! WHEN (vol(bar) = vol(NULL)) THEN null input WHEN (vol(bar) = bar) THEN it was bar! END = it was foo!) THEN foo recognized WHEN (CASE WHEN (vol(bar) = foo) THEN it was foo! WHEN (vol(bar) = vol(NULL)) THEN null input WHEN (vol(bar) = bar) THEN it was bar! END = it was bar!) THEN bar recognized ELSE unrecognized END#x] ++- OneRowRelation + + +-- !query +DROP TABLE CASE_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`CASE_TBL`, false, false, false + + +-- !query +DROP TABLE CASE2_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`CASE2_TBL`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/comments.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/comments.sql.out new file mode 100644 index 0000000000000..f932999f2aca6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/comments.sql.out @@ -0,0 +1,65 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT 'trailing' AS first +-- !query analysis +Project [trailing AS first#x] ++- OneRowRelation + + +-- !query +SELECT /* embedded single line */ 'embedded' AS `second` +-- !query analysis +Project [embedded AS second#x] ++- OneRowRelation + + +-- !query +SELECT /* both embedded and trailing single line */ 'both' AS third +-- !query analysis +Project [both AS third#x] ++- OneRowRelation + + +-- !query +SELECT 'before multi-line' AS fourth +-- !query analysis +Project [before multi-line AS fourth#x] ++- OneRowRelation + + +-- !query +/* This is an example of SQL which should not execute: + * select 'multi-line'; + */ +SELECT 'after multi-line' AS fifth +-- !query analysis +Project [after multi-line AS fifth#x] ++- OneRowRelation + + +-- !query +/* +SELECT 'trailing' as x1; -- inside block comment +*/ + +/* This block comment surrounds a query which itself has a block comment... +SELECT /* embedded single line */ 'embedded' AS x2; +*/ + +SELECT -- continued after the following block comments... +/* Deeply nested comment. + This includes a single apostrophe to make sure we aren't decoding this part as a string. +SELECT 'deep nest' AS n1; +/* Second level of nesting... +SELECT 'deeper nest' as n2; +/* Third level of nesting... +SELECT 'deepest nest' as n3; +*/ +Hoo boy. Still two deep... +*/ +Now just one deep... +*/ +'deeply nested example' AS sixth +-- !query analysis +Project [deeply nested example AS sixth#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out new file mode 100644 index 0000000000000..c05ee8dc6b464 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out @@ -0,0 +1,1899 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE emp ( + name string, + age int, + salary int, + manager string +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`emp`, false + + +-- !query +CREATE VIEW toyemp AS + SELECT name, age, /* location ,*/ 12*salary AS annualsal + FROM emp +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`toyemp`, SELECT name, age, /* location ,*/ 12*salary AS annualsal + FROM emp, false, false, PersistedView, true + +- Project [name#x, age#x, (12 * salary#x) AS annualsal#x] + +- SubqueryAlias spark_catalog.default.emp + +- Relation spark_catalog.default.emp[name#x,age#x,salary#x,manager#x] parquet + + +-- !query +DROP VIEW toyemp +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`toyemp`, false, true, false + + +-- !query +DROP TABLE emp +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`emp`, false, false, false + + +-- !query +CREATE TABLE view_base_table (key int /* PRIMARY KEY */, data varchar(20)) USING PARQUET +-- !query analysis +org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +{ + "errorClass" : "TABLE_OR_VIEW_ALREADY_EXISTS", + "sqlState" : "42P07", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`view_base_table`" + } +} + + +-- !query +CREATE VIEW key_dependent_view AS + SELECT * FROM view_base_table GROUP BY key +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"data\"", + "expressionAnyValue" : "\"any_value(data)\"" + } +} + + +-- !query +CREATE VIEW key_dependent_view_no_cols AS + SELECT FROM view_base_table GROUP BY key HAVING length(data) > 0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`FROM`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 53, + "stopIndex" : 56, + "fragment" : "FROM" + } ] +} + + +-- !query +CREATE TABLE viewtest_tbl (a int, b int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`viewtest_tbl`, false + + +-- !query +INSERT INTO viewtest_tbl VALUES (5, 10), (10, 15), (15, 20), (20, 25) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/viewtest_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/viewtest_tbl], Append, `spark_catalog`.`default`.`viewtest_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/viewtest_tbl), [a, b] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE OR REPLACE VIEW viewtest AS + SELECT * FROM viewtest_tbl +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`viewtest`, SELECT * FROM viewtest_tbl, false, true, PersistedView, true + +- Project [a#x, b#x] + +- SubqueryAlias spark_catalog.default.viewtest_tbl + +- Relation spark_catalog.default.viewtest_tbl[a#x,b#x] parquet + + +-- !query +CREATE OR REPLACE VIEW viewtest AS + SELECT * FROM viewtest_tbl WHERE a > 10 +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`viewtest`, SELECT * FROM viewtest_tbl WHERE a > 10, false, true, PersistedView, true + +- Project [a#x, b#x] + +- Filter (a#x > 10) + +- SubqueryAlias spark_catalog.default.viewtest_tbl + +- Relation spark_catalog.default.viewtest_tbl[a#x,b#x] parquet + + +-- !query +SELECT * FROM viewtest +-- !query analysis +Project [a#x, b#x] ++- SubqueryAlias spark_catalog.default.viewtest + +- View (`spark_catalog`.`default`.`viewtest`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- Filter (a#x > 10) + +- SubqueryAlias spark_catalog.default.viewtest_tbl + +- Relation spark_catalog.default.viewtest_tbl[a#x,b#x] parquet + + +-- !query +CREATE OR REPLACE VIEW viewtest AS + SELECT a, b FROM viewtest_tbl WHERE a > 5 ORDER BY b DESC +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`viewtest`, SELECT a, b FROM viewtest_tbl WHERE a > 5 ORDER BY b DESC, false, true, PersistedView, true + +- Sort [b#x DESC NULLS LAST], true + +- Project [a#x, b#x] + +- Filter (a#x > 5) + +- SubqueryAlias spark_catalog.default.viewtest_tbl + +- Relation spark_catalog.default.viewtest_tbl[a#x,b#x] parquet + + +-- !query +SELECT * FROM viewtest +-- !query analysis +Project [a#x, b#x] ++- SubqueryAlias spark_catalog.default.viewtest + +- View (`spark_catalog`.`default`.`viewtest`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Sort [b#x DESC NULLS LAST], true + +- Project [a#x, b#x] + +- Filter (a#x > 5) + +- SubqueryAlias spark_catalog.default.viewtest_tbl + +- Relation spark_catalog.default.viewtest_tbl[a#x,b#x] parquet + + +-- !query +CREATE OR REPLACE VIEW viewtest AS + SELECT a FROM viewtest_tbl WHERE a <> 20 +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`viewtest`, SELECT a FROM viewtest_tbl WHERE a <> 20, false, true, PersistedView, true + +- Project [a#x] + +- Filter NOT (a#x = 20) + +- SubqueryAlias spark_catalog.default.viewtest_tbl + +- Relation spark_catalog.default.viewtest_tbl[a#x,b#x] parquet + + +-- !query +CREATE OR REPLACE VIEW viewtest AS + SELECT 1, * FROM viewtest_tbl +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`viewtest`, SELECT 1, * FROM viewtest_tbl, false, true, PersistedView, true + +- Project [1 AS 1#x, a#x, b#x] + +- SubqueryAlias spark_catalog.default.viewtest_tbl + +- Relation spark_catalog.default.viewtest_tbl[a#x,b#x] parquet + + +-- !query +CREATE OR REPLACE VIEW viewtest AS + SELECT a, decimal(b) FROM viewtest_tbl +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`viewtest`, SELECT a, decimal(b) FROM viewtest_tbl, false, true, PersistedView, true + +- Project [a#x, cast(b#x as decimal(10,0)) AS b#x] + +- SubqueryAlias spark_catalog.default.viewtest_tbl + +- Relation spark_catalog.default.viewtest_tbl[a#x,b#x] parquet + + +-- !query +CREATE OR REPLACE VIEW viewtest AS + SELECT a, b, 0 AS c FROM viewtest_tbl +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`viewtest`, SELECT a, b, 0 AS c FROM viewtest_tbl, false, true, PersistedView, true + +- Project [a#x, b#x, 0 AS c#x] + +- SubqueryAlias spark_catalog.default.viewtest_tbl + +- Relation spark_catalog.default.viewtest_tbl[a#x,b#x] parquet + + +-- !query +DROP VIEW viewtest +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`viewtest`, false, true, false + + +-- !query +DROP TABLE viewtest_tbl +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`viewtest_tbl`, false, false, false + + +-- !query +CREATE SCHEMA temp_view_test +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [temp_view_test] + + +-- !query +CREATE TABLE temp_view_test.base_table (a int, id int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`temp_view_test`.`base_table`, false + + +-- !query +CREATE TABLE temp_view_test.base_table2 (a int, id int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`temp_view_test`.`base_table2`, false + + +-- !query +USE temp_view_test +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [temp_view_test] + + +-- !query +CREATE TEMPORARY VIEW temp_table AS SELECT * FROM VALUES + (1, 1) as temp_table(a, id) +-- !query analysis +CreateViewCommand `temp_table`, SELECT * FROM VALUES + (1, 1) as temp_table(a, id), false, false, LocalTempView, true + +- Project [a#x, id#x] + +- SubqueryAlias temp_table + +- LocalRelation [a#x, id#x] + + +-- !query +CREATE VIEW v1 AS SELECT * FROM base_table +-- !query analysis +CreateViewCommand `spark_catalog`.`temp_view_test`.`v1`, SELECT * FROM base_table, false, false, PersistedView, true + +- Project [a#x, id#x] + +- SubqueryAlias spark_catalog.temp_view_test.base_table + +- Relation spark_catalog.temp_view_test.base_table[a#x,id#x] parquet + + +-- !query +DESC TABLE EXTENDED v1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`temp_view_test`.`v1`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE VIEW v1_temp AS SELECT * FROM temp_table +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v1_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE TEMP VIEW v2_temp AS SELECT * FROM base_table +-- !query analysis +CreateViewCommand `v2_temp`, SELECT * FROM base_table, false, false, LocalTempView, true + +- Project [a#x, id#x] + +- SubqueryAlias spark_catalog.temp_view_test.base_table + +- Relation spark_catalog.temp_view_test.base_table[a#x,id#x] parquet + + +-- !query +DESC TABLE EXTENDED v2_temp +-- !query analysis +DescribeTableCommand `v2_temp`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE VIEW temp_view_test.v2 AS SELECT * FROM base_table +-- !query analysis +CreateViewCommand `spark_catalog`.`temp_view_test`.`v2`, SELECT * FROM base_table, false, false, PersistedView, true + +- Project [a#x, id#x] + +- SubqueryAlias spark_catalog.temp_view_test.base_table + +- Relation spark_catalog.temp_view_test.base_table[a#x,id#x] parquet + + +-- !query +DESC TABLE EXTENDED temp_view_test.v2 +-- !query analysis +DescribeTableCommand `spark_catalog`.`temp_view_test`.`v2`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE VIEW temp_view_test.v3_temp AS SELECT * FROM temp_table +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v3_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE VIEW v3 AS + SELECT t1.a AS t1_a, t2.a AS t2_a + FROM base_table t1, base_table2 t2 + WHERE t1.id = t2.id +-- !query analysis +CreateViewCommand `spark_catalog`.`temp_view_test`.`v3`, SELECT t1.a AS t1_a, t2.a AS t2_a + FROM base_table t1, base_table2 t2 + WHERE t1.id = t2.id, false, false, PersistedView, true + +- Project [a#x AS t1_a#x, a#x AS t2_a#x] + +- Filter (id#x = id#x) + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.temp_view_test.base_table + : +- Relation spark_catalog.temp_view_test.base_table[a#x,id#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.temp_view_test.base_table2 + +- Relation spark_catalog.temp_view_test.base_table2[a#x,id#x] parquet + + +-- !query +DESC TABLE EXTENDED v3 +-- !query analysis +DescribeTableCommand `spark_catalog`.`temp_view_test`.`v3`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE VIEW v4_temp AS + SELECT t1.a AS t1_a, t2.a AS t2_a + FROM base_table t1, temp_table t2 + WHERE t1.id = t2.id +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v4_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE VIEW v5_temp AS + SELECT t1.a AS t1_a, t2.a AS t2_a, t3.a AS t3_a + FROM base_table t1, base_table2 t2, temp_table t3 + WHERE t1.id = t2.id and t2.id = t3.id +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v5_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE VIEW v4 AS SELECT * FROM base_table WHERE id IN (SELECT id FROM base_table2) +-- !query analysis +CreateViewCommand `spark_catalog`.`temp_view_test`.`v4`, SELECT * FROM base_table WHERE id IN (SELECT id FROM base_table2), false, false, PersistedView, true + +- Project [a#x, id#x] + +- Filter id#x IN (list#x []) + : +- Project [id#x] + : +- SubqueryAlias spark_catalog.temp_view_test.base_table2 + : +- Relation spark_catalog.temp_view_test.base_table2[a#x,id#x] parquet + +- SubqueryAlias spark_catalog.temp_view_test.base_table + +- Relation spark_catalog.temp_view_test.base_table[a#x,id#x] parquet + + +-- !query +DESC TABLE EXTENDED v4 +-- !query analysis +DescribeTableCommand `spark_catalog`.`temp_view_test`.`v4`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE VIEW v5 AS SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM base_table2) t2 +-- !query analysis +CreateViewCommand `spark_catalog`.`temp_view_test`.`v5`, SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM base_table2) t2, false, false, PersistedView, true + +- Project [id#x, a#x] + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.temp_view_test.base_table + : +- Relation spark_catalog.temp_view_test.base_table[a#x,id#x] parquet + +- SubqueryAlias t2 + +- Project [a#x, id#x] + +- SubqueryAlias spark_catalog.temp_view_test.base_table2 + +- Relation spark_catalog.temp_view_test.base_table2[a#x,id#x] parquet + + +-- !query +DESC TABLE EXTENDED v5 +-- !query analysis +DescribeTableCommand `spark_catalog`.`temp_view_test`.`v5`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE VIEW v6 AS SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM base_table2) +-- !query analysis +CreateViewCommand `spark_catalog`.`temp_view_test`.`v6`, SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM base_table2), false, false, PersistedView, true + +- Project [a#x, id#x] + +- Filter exists#x [] + : +- Project [1 AS 1#x] + : +- SubqueryAlias spark_catalog.temp_view_test.base_table2 + : +- Relation spark_catalog.temp_view_test.base_table2[a#x,id#x] parquet + +- SubqueryAlias spark_catalog.temp_view_test.base_table + +- Relation spark_catalog.temp_view_test.base_table[a#x,id#x] parquet + + +-- !query +DESC TABLE EXTENDED v6 +-- !query analysis +DescribeTableCommand `spark_catalog`.`temp_view_test`.`v6`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE VIEW v7 AS SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM base_table2) +-- !query analysis +CreateViewCommand `spark_catalog`.`temp_view_test`.`v7`, SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM base_table2), false, false, PersistedView, true + +- Project [a#x, id#x] + +- Filter NOT exists#x [] + : +- Project [1 AS 1#x] + : +- SubqueryAlias spark_catalog.temp_view_test.base_table2 + : +- Relation spark_catalog.temp_view_test.base_table2[a#x,id#x] parquet + +- SubqueryAlias spark_catalog.temp_view_test.base_table + +- Relation spark_catalog.temp_view_test.base_table[a#x,id#x] parquet + + +-- !query +DESC TABLE EXTENDED v7 +-- !query analysis +DescribeTableCommand `spark_catalog`.`temp_view_test`.`v7`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE VIEW v8 AS SELECT * FROM base_table WHERE EXISTS (SELECT 1) +-- !query analysis +CreateViewCommand `spark_catalog`.`temp_view_test`.`v8`, SELECT * FROM base_table WHERE EXISTS (SELECT 1), false, false, PersistedView, true + +- Project [a#x, id#x] + +- Filter exists#x [] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- SubqueryAlias spark_catalog.temp_view_test.base_table + +- Relation spark_catalog.temp_view_test.base_table[a#x,id#x] parquet + + +-- !query +DESC TABLE EXTENDED v8 +-- !query analysis +DescribeTableCommand `spark_catalog`.`temp_view_test`.`v8`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE VIEW v6_temp AS SELECT * FROM base_table WHERE id IN (SELECT id FROM temp_table) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v6_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE VIEW v7_temp AS SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM temp_table) t2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v7_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE VIEW v8_temp AS SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM temp_table) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v8_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE VIEW v9_temp AS SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM temp_table) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v9_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE VIEW v10_temp AS SELECT * FROM v7_temp +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`v7_temp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 45, + "fragment" : "v7_temp" + } ] +} + + +-- !query +CREATE VIEW v11_temp AS SELECT t1.id, t2.a FROM base_table t1, v10_temp t2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`v10_temp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 64, + "stopIndex" : 74, + "fragment" : "v10_temp t2" + } ] +} + + +-- !query +CREATE VIEW v12_temp AS SELECT true FROM v11_temp +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`v11_temp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 42, + "stopIndex" : 49, + "fragment" : "v11_temp" + } ] +} + + +-- !query +CREATE SCHEMA testviewschm2 +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [testviewschm2] + + +-- !query +USE testviewschm2 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [testviewschm2] + + +-- !query +CREATE TABLE t1 (num int, name string) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`t1`, false + + +-- !query +CREATE TABLE t2 (num2 int, value string) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`t2`, false + + +-- !query +CREATE TEMP VIEW tt AS SELECT * FROM VALUES + (1, 'a') AS tt(num2, value) +-- !query analysis +CreateViewCommand `tt`, SELECT * FROM VALUES + (1, 'a') AS tt(num2, value), false, false, LocalTempView, true + +- Project [num2#x, value#x] + +- SubqueryAlias tt + +- LocalRelation [num2#x, value#x] + + +-- !query +CREATE VIEW nontemp1 AS SELECT * FROM t1 CROSS JOIN t2 +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`nontemp1`, SELECT * FROM t1 CROSS JOIN t2, false, false, PersistedView, true + +- Project [num#x, name#x, num2#x, value#x] + +- Join Cross + :- SubqueryAlias spark_catalog.testviewschm2.t1 + : +- Relation spark_catalog.testviewschm2.t1[num#x,name#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.t2 + +- Relation spark_catalog.testviewschm2.t2[num2#x,value#x] parquet + + +-- !query +DESC TABLE EXTENDED nontemp1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`nontemp1`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE VIEW temporal1 AS SELECT * FROM t1 CROSS JOIN tt +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`testviewschm2`.`temporal1`", + "tempObj" : "VIEW", + "tempObjName" : "`tt`" + } +} + + +-- !query +CREATE VIEW nontemp2 AS SELECT * FROM t1 INNER JOIN t2 ON t1.num = t2.num2 +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`nontemp2`, SELECT * FROM t1 INNER JOIN t2 ON t1.num = t2.num2, false, false, PersistedView, true + +- Project [num#x, name#x, num2#x, value#x] + +- Join Inner, (num#x = num2#x) + :- SubqueryAlias spark_catalog.testviewschm2.t1 + : +- Relation spark_catalog.testviewschm2.t1[num#x,name#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.t2 + +- Relation spark_catalog.testviewschm2.t2[num2#x,value#x] parquet + + +-- !query +DESC TABLE EXTENDED nontemp2 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`nontemp2`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE VIEW temporal2 AS SELECT * FROM t1 INNER JOIN tt ON t1.num = tt.num2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`testviewschm2`.`temporal2`", + "tempObj" : "VIEW", + "tempObjName" : "`tt`" + } +} + + +-- !query +CREATE VIEW nontemp3 AS SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`nontemp3`, SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2, false, false, PersistedView, true + +- Project [num#x, name#x, num2#x, value#x] + +- Join LeftOuter, (num#x = num2#x) + :- SubqueryAlias spark_catalog.testviewschm2.t1 + : +- Relation spark_catalog.testviewschm2.t1[num#x,name#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.t2 + +- Relation spark_catalog.testviewschm2.t2[num2#x,value#x] parquet + + +-- !query +DESC TABLE EXTENDED nontemp3 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`nontemp3`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE VIEW temporal3 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`testviewschm2`.`temporal3`", + "tempObj" : "VIEW", + "tempObjName" : "`tt`" + } +} + + +-- !query +CREATE VIEW nontemp4 AS SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 AND t2.value = 'xxx' +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`nontemp4`, SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 AND t2.value = 'xxx', false, false, PersistedView, true + +- Project [num#x, name#x, num2#x, value#x] + +- Join LeftOuter, ((num#x = num2#x) AND (value#x = xxx)) + :- SubqueryAlias spark_catalog.testviewschm2.t1 + : +- Relation spark_catalog.testviewschm2.t1[num#x,name#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.t2 + +- Relation spark_catalog.testviewschm2.t2[num2#x,value#x] parquet + + +-- !query +DESC TABLE EXTENDED nontemp4 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`nontemp4`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE VIEW temporal4 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2 AND tt.value = 'xxx' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`testviewschm2`.`temporal4`", + "tempObj" : "VIEW", + "tempObjName" : "`tt`" + } +} + + +-- !query +CREATE VIEW temporal5 AS SELECT * FROM t1 WHERE num IN (SELECT num FROM t1 WHERE EXISTS (SELECT 1 FROM tt)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`testviewschm2`.`temporal5`", + "tempObj" : "VIEW", + "tempObjName" : "`tt`" + } +} + + +-- !query +CREATE TABLE tbl1 ( a int, b int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tbl1`, false + + +-- !query +CREATE TABLE tbl2 (c int, d int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tbl2`, false + + +-- !query +CREATE TABLE tbl3 (e int, f int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tbl3`, false + + +-- !query +CREATE TABLE tbl4 (g int, h int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tbl4`, false + + +-- !query +CREATE TABLE tmptbl (i int, j int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tmptbl`, false + + +-- !query +INSERT INTO tmptbl VALUES (1, 1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/testviewschm2.db/tmptbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/testviewschm2.db/tmptbl], Append, `spark_catalog`.`testviewschm2`.`tmptbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/testviewschm2.db/tmptbl), [i, j] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE VIEW pubview AS SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`pubview`, SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f), false, false, PersistedView, true + +- Project [a#x, b#x] + +- Filter (((a#x >= scalar-subquery#x []) AND (a#x <= scalar-subquery#x [])) AND exists#x []) + : :- Project [d#x] + : : +- Filter (c#x = 1) + : : +- SubqueryAlias spark_catalog.testviewschm2.tbl2 + : : +- Relation spark_catalog.testviewschm2.tbl2[c#x,d#x] parquet + : :- Project [e#x] + : : +- Filter (f#x = 2) + : : +- SubqueryAlias spark_catalog.testviewschm2.tbl3 + : : +- Relation spark_catalog.testviewschm2.tbl3[e#x,f#x] parquet + : +- Project [g#x] + : +- Join LeftOuter, (h#x = f#x) + : :- SubqueryAlias spark_catalog.testviewschm2.tbl4 + : : +- Relation spark_catalog.testviewschm2.tbl4[g#x,h#x] parquet + : +- SubqueryAlias spark_catalog.testviewschm2.tbl3 + : +- Relation spark_catalog.testviewschm2.tbl3[e#x,f#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.tbl1 + +- Relation spark_catalog.testviewschm2.tbl1[a#x,b#x] parquet + + +-- !query +DESC TABLE EXTENDED pubview +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`pubview`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE VIEW mytempview AS SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +AND NOT EXISTS (SELECT g FROM tbl4 LEFT JOIN tmptbl ON tbl4.h = tmptbl.j) +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`mytempview`, SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +AND NOT EXISTS (SELECT g FROM tbl4 LEFT JOIN tmptbl ON tbl4.h = tmptbl.j), false, false, PersistedView, true + +- Project [a#x, b#x] + +- Filter ((((a#x >= scalar-subquery#x []) AND (a#x <= scalar-subquery#x [])) AND exists#x []) AND NOT exists#x []) + : :- Project [d#x] + : : +- Filter (c#x = 1) + : : +- SubqueryAlias spark_catalog.testviewschm2.tbl2 + : : +- Relation spark_catalog.testviewschm2.tbl2[c#x,d#x] parquet + : :- Project [e#x] + : : +- Filter (f#x = 2) + : : +- SubqueryAlias spark_catalog.testviewschm2.tbl3 + : : +- Relation spark_catalog.testviewschm2.tbl3[e#x,f#x] parquet + : :- Project [g#x] + : : +- Join LeftOuter, (h#x = f#x) + : : :- SubqueryAlias spark_catalog.testviewschm2.tbl4 + : : : +- Relation spark_catalog.testviewschm2.tbl4[g#x,h#x] parquet + : : +- SubqueryAlias spark_catalog.testviewschm2.tbl3 + : : +- Relation spark_catalog.testviewschm2.tbl3[e#x,f#x] parquet + : +- Project [g#x] + : +- Join LeftOuter, (h#x = j#x) + : :- SubqueryAlias spark_catalog.testviewschm2.tbl4 + : : +- Relation spark_catalog.testviewschm2.tbl4[g#x,h#x] parquet + : +- SubqueryAlias spark_catalog.testviewschm2.tmptbl + : +- Relation spark_catalog.testviewschm2.tmptbl[i#x,j#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.tbl1 + +- Relation spark_catalog.testviewschm2.tbl1[a#x,b#x] parquet + + +-- !query +DESC TABLE EXTENDED mytempview +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`mytempview`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE VIEW tt1 AS + SELECT * FROM ( + VALUES + ('abc', '0123456789', 42, 'abcd'), + ('0123456789', 'abc', 42.12, 'abc') + ) vv(a,b,c,d) +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`tt1`, SELECT * FROM ( + VALUES + ('abc', '0123456789', 42, 'abcd'), + ('0123456789', 'abc', 42.12, 'abc') + ) vv(a,b,c,d), false, false, PersistedView, true + +- Project [a#x, b#x, c#x, d#x] + +- SubqueryAlias vv + +- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +SELECT * FROM tt1 +-- !query analysis +Project [a#x, b#x, c#x, d#x] ++- SubqueryAlias spark_catalog.testviewschm2.tt1 + +- View (`spark_catalog`.`testviewschm2`.`tt1`, [a#x,b#x,c#x,d#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x, cast(c#x as decimal(12,2)) AS c#x, cast(d#x as string) AS d#x] + +- Project [a#x, b#x, c#x, d#x] + +- SubqueryAlias vv + +- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +SELECT string(a) FROM tt1 +-- !query analysis +Project [cast(a#x as string) AS a#x] ++- SubqueryAlias spark_catalog.testviewschm2.tt1 + +- View (`spark_catalog`.`testviewschm2`.`tt1`, [a#x,b#x,c#x,d#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x, cast(c#x as decimal(12,2)) AS c#x, cast(d#x as string) AS d#x] + +- Project [a#x, b#x, c#x, d#x] + +- SubqueryAlias vv + +- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +DROP VIEW tt1 +-- !query analysis +DropTableCommand `spark_catalog`.`testviewschm2`.`tt1`, false, true, false + + +-- !query +CREATE TABLE tt1 (f1 int, f2 int, f3 string) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt1`, false + + +-- !query +CREATE TABLE tx1 (x1 int, x2 int, x3 string) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tx1`, false + + +-- !query +CREATE TABLE temp_view_test.tt1 (y1 int, f2 int, f3 string) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`temp_view_test`.`tt1`, false + + +-- !query +CREATE VIEW aliased_view_1 AS + select * from tt1 + where exists (select 1 from tx1 where tt1.f1 = tx1.x1) +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`aliased_view_1`, select * from tt1 + where exists (select 1 from tx1 where tt1.f1 = tx1.x1), false, false, PersistedView, true + +- Project [f1#x, f2#x, f3#x] + +- Filter exists#x [f1#x] + : +- Project [1 AS 1#x] + : +- Filter (outer(f1#x) = x1#x) + : +- SubqueryAlias spark_catalog.testviewschm2.tx1 + : +- Relation spark_catalog.testviewschm2.tx1[x1#x,x2#x,x3#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.tt1 + +- Relation spark_catalog.testviewschm2.tt1[f1#x,f2#x,f3#x] parquet + + +-- !query +CREATE VIEW aliased_view_2 AS + select * from tt1 a1 + where exists (select 1 from tx1 where a1.f1 = tx1.x1) +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`aliased_view_2`, select * from tt1 a1 + where exists (select 1 from tx1 where a1.f1 = tx1.x1), false, false, PersistedView, true + +- Project [f1#x, f2#x, f3#x] + +- Filter exists#x [f1#x] + : +- Project [1 AS 1#x] + : +- Filter (outer(f1#x) = x1#x) + : +- SubqueryAlias spark_catalog.testviewschm2.tx1 + : +- Relation spark_catalog.testviewschm2.tx1[x1#x,x2#x,x3#x] parquet + +- SubqueryAlias a1 + +- SubqueryAlias spark_catalog.testviewschm2.tt1 + +- Relation spark_catalog.testviewschm2.tt1[f1#x,f2#x,f3#x] parquet + + +-- !query +CREATE VIEW aliased_view_3 AS + select * from tt1 + where exists (select 1 from tx1 a2 where tt1.f1 = a2.x1) +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`aliased_view_3`, select * from tt1 + where exists (select 1 from tx1 a2 where tt1.f1 = a2.x1), false, false, PersistedView, true + +- Project [f1#x, f2#x, f3#x] + +- Filter exists#x [f1#x] + : +- Project [1 AS 1#x] + : +- Filter (outer(f1#x) = x1#x) + : +- SubqueryAlias a2 + : +- SubqueryAlias spark_catalog.testviewschm2.tx1 + : +- Relation spark_catalog.testviewschm2.tx1[x1#x,x2#x,x3#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.tt1 + +- Relation spark_catalog.testviewschm2.tt1[f1#x,f2#x,f3#x] parquet + + +-- !query +CREATE VIEW aliased_view_4 AS + select * from temp_view_test.tt1 + where exists (select 1 from tt1 where temp_view_test.tt1.y1 = tt1.f1) +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`aliased_view_4`, select * from temp_view_test.tt1 + where exists (select 1 from tt1 where temp_view_test.tt1.y1 = tt1.f1), false, false, PersistedView, true + +- Project [y1#x, f2#x, f3#x] + +- Filter exists#x [y1#x] + : +- Project [1 AS 1#x] + : +- Filter (outer(y1#x) = f1#x) + : +- SubqueryAlias spark_catalog.testviewschm2.tt1 + : +- Relation spark_catalog.testviewschm2.tt1[f1#x,f2#x,f3#x] parquet + +- SubqueryAlias spark_catalog.temp_view_test.tt1 + +- Relation spark_catalog.temp_view_test.tt1[y1#x,f2#x,f3#x] parquet + + +-- !query +DESC TABLE aliased_view_1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_1`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE aliased_view_2 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_2`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE aliased_view_3 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_3`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE aliased_view_4 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_4`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE tx1 RENAME TO a1 +-- !query analysis +AlterTableRenameCommand `spark_catalog`.`testviewschm2`.`tx1`, `a1`, false + + +-- !query +DESC TABLE aliased_view_1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_1`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE aliased_view_2 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_2`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE aliased_view_3 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_3`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE aliased_view_4 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_4`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE tt1 RENAME TO a2 +-- !query analysis +AlterTableRenameCommand `spark_catalog`.`testviewschm2`.`tt1`, `a2`, false + + +-- !query +DESC TABLE aliased_view_1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_1`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE aliased_view_2 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_2`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE aliased_view_3 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_3`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE aliased_view_4 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_4`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE a1 RENAME TO tt1 +-- !query analysis +AlterTableRenameCommand `spark_catalog`.`testviewschm2`.`a1`, `tt1`, false + + +-- !query +DESC TABLE aliased_view_1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_1`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE aliased_view_2 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_2`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE aliased_view_3 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_3`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE aliased_view_4 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`aliased_view_4`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +ALTER TABLE a2 RENAME TO tx1 +-- !query analysis +AlterTableRenameCommand `spark_catalog`.`testviewschm2`.`a2`, `tx1`, false + + +-- !query +create view view_of_joins as +select * from + (select * from (tbl1 cross join tbl2) same) ss, + (tbl3 cross join tbl4) same +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`view_of_joins`, select * from + (select * from (tbl1 cross join tbl2) same) ss, + (tbl3 cross join tbl4) same, false, false, PersistedView, true + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x] + +- Join Inner + :- SubqueryAlias ss + : +- Project [a#x, b#x, c#x, d#x] + : +- SubqueryAlias same + : +- Join Cross + : :- SubqueryAlias spark_catalog.testviewschm2.tbl1 + : : +- Relation spark_catalog.testviewschm2.tbl1[a#x,b#x] parquet + : +- SubqueryAlias spark_catalog.testviewschm2.tbl2 + : +- Relation spark_catalog.testviewschm2.tbl2[c#x,d#x] parquet + +- SubqueryAlias same + +- Join Cross + :- SubqueryAlias spark_catalog.testviewschm2.tbl3 + : +- Relation spark_catalog.testviewschm2.tbl3[e#x,f#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.tbl4 + +- Relation spark_catalog.testviewschm2.tbl4[g#x,h#x] parquet + + +-- !query +create table tt2 (a int, b int, c int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt2`, false + + +-- !query +create table tt3 (ax bigint, b short, c decimal) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt3`, false + + +-- !query +create table tt4 (ay int, b int, q int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt4`, false + + +-- !query +create view v1 as select * from tt2 natural join tt3 +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`v1`, select * from tt2 natural join tt3, false, false, PersistedView, true + +- Project [b#x, c#x, a#x, ax#xL] + +- Project [b#x, c#x, a#x, ax#xL] + +- Join Inner, ((b#x = cast(b#x as int)) AND (cast(c#x as decimal(10,0)) = c#x)) + :- SubqueryAlias spark_catalog.testviewschm2.tt2 + : +- Relation spark_catalog.testviewschm2.tt2[a#x,b#x,c#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.tt3 + +- Relation spark_catalog.testviewschm2.tt3[ax#xL,b#x,c#x] parquet + + +-- !query +create view v1a as select * from (tt2 natural join tt3) j +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`v1a`, select * from (tt2 natural join tt3) j, false, false, PersistedView, true + +- Project [b#x, c#x, a#x, ax#xL] + +- SubqueryAlias j + +- Project [b#x, c#x, a#x, ax#xL] + +- Join Inner, ((b#x = cast(b#x as int)) AND (cast(c#x as decimal(10,0)) = c#x)) + :- SubqueryAlias spark_catalog.testviewschm2.tt2 + : +- Relation spark_catalog.testviewschm2.tt2[a#x,b#x,c#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.tt3 + +- Relation spark_catalog.testviewschm2.tt3[ax#xL,b#x,c#x] parquet + + +-- !query +create view v2 as select * from tt2 join tt3 using (b,c) join tt4 using (b) +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`v2`, select * from tt2 join tt3 using (b,c) join tt4 using (b), false, false, PersistedView, true + +- Project [b#x, c#x, a#x, ax#xL, ay#x, q#x] + +- Project [b#x, c#x, a#x, ax#xL, ay#x, q#x] + +- Join Inner, (b#x = b#x) + :- Project [b#x, c#x, a#x, ax#xL] + : +- Join Inner, ((b#x = cast(b#x as int)) AND (cast(c#x as decimal(10,0)) = c#x)) + : :- SubqueryAlias spark_catalog.testviewschm2.tt2 + : : +- Relation spark_catalog.testviewschm2.tt2[a#x,b#x,c#x] parquet + : +- SubqueryAlias spark_catalog.testviewschm2.tt3 + : +- Relation spark_catalog.testviewschm2.tt3[ax#xL,b#x,c#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.tt4 + +- Relation spark_catalog.testviewschm2.tt4[ay#x,b#x,q#x] parquet + + +-- !query +create view v2a as select * from (tt2 join tt3 using (b,c) join tt4 using (b)) j +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`v2a`, select * from (tt2 join tt3 using (b,c) join tt4 using (b)) j, false, false, PersistedView, true + +- Project [b#x, c#x, a#x, ax#xL, ay#x, q#x] + +- SubqueryAlias j + +- Project [b#x, c#x, a#x, ax#xL, ay#x, q#x] + +- Join Inner, (b#x = b#x) + :- Project [b#x, c#x, a#x, ax#xL] + : +- Join Inner, ((b#x = cast(b#x as int)) AND (cast(c#x as decimal(10,0)) = c#x)) + : :- SubqueryAlias spark_catalog.testviewschm2.tt2 + : : +- Relation spark_catalog.testviewschm2.tt2[a#x,b#x,c#x] parquet + : +- SubqueryAlias spark_catalog.testviewschm2.tt3 + : +- Relation spark_catalog.testviewschm2.tt3[ax#xL,b#x,c#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.tt4 + +- Relation spark_catalog.testviewschm2.tt4[ay#x,b#x,q#x] parquet + + +-- !query +create view v3 as select * from tt2 join tt3 using (b,c) full join tt4 using (b) +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`v3`, select * from tt2 join tt3 using (b,c) full join tt4 using (b), false, false, PersistedView, true + +- Project [b#x, c#x, a#x, ax#xL, ay#x, q#x] + +- Project [coalesce(b#x, b#x) AS b#x, c#x, a#x, ax#xL, ay#x, q#x] + +- Join FullOuter, (b#x = b#x) + :- Project [b#x, c#x, a#x, ax#xL] + : +- Join Inner, ((b#x = cast(b#x as int)) AND (cast(c#x as decimal(10,0)) = c#x)) + : :- SubqueryAlias spark_catalog.testviewschm2.tt2 + : : +- Relation spark_catalog.testviewschm2.tt2[a#x,b#x,c#x] parquet + : +- SubqueryAlias spark_catalog.testviewschm2.tt3 + : +- Relation spark_catalog.testviewschm2.tt3[ax#xL,b#x,c#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.tt4 + +- Relation spark_catalog.testviewschm2.tt4[ay#x,b#x,q#x] parquet + + +-- !query +DESC TABLE v1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`v1`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE v1a +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`v1a`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE v2 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`v2`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE v2a +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`v2a`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE v3 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`v3`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +alter table tt2 add column d int +-- !query analysis +AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt2`, [StructField(d,IntegerType,true)] + + +-- !query +alter table tt2 add column e int +-- !query analysis +AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt2`, [StructField(e,IntegerType,true)] + + +-- !query +DESC TABLE v1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`v1`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE v1a +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`v1a`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE v2 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`v2`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE v2a +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`v2a`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE v3 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`v3`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +drop table tt3 +-- !query analysis +DropTableCommand `spark_catalog`.`testviewschm2`.`tt3`, false, false, false + + +-- !query +create table tt3 (ax bigint, b short, d decimal) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt3`, false + + +-- !query +alter table tt3 add column c int +-- !query analysis +AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt3`, [StructField(c,IntegerType,true)] + + +-- !query +alter table tt3 add column e int +-- !query analysis +AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt3`, [StructField(e,IntegerType,true)] + + +-- !query +DESC TABLE v1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`v1`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE v1a +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`v1a`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE v2 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`v2`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE v2a +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`v2a`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE v3 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`v3`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +create table tt5 (a int, b int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt5`, false + + +-- !query +create table tt6 (c int, d int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt6`, false + + +-- !query +create view vv1 as select * from (tt5 cross join tt6) j(aa,bb,cc,dd) +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`vv1`, select * from (tt5 cross join tt6) j(aa,bb,cc,dd), false, false, PersistedView, true + +- Project [aa#x, bb#x, cc#x, dd#x] + +- SubqueryAlias j + +- Project [a#x AS aa#x, b#x AS bb#x, c#x AS cc#x, d#x AS dd#x] + +- Join Cross + :- SubqueryAlias spark_catalog.testviewschm2.tt5 + : +- Relation spark_catalog.testviewschm2.tt5[a#x,b#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.tt6 + +- Relation spark_catalog.testviewschm2.tt6[c#x,d#x] parquet + + +-- !query +DESC TABLE vv1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv1`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +alter table tt5 add column c int +-- !query analysis +AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt5`, [StructField(c,IntegerType,true)] + + +-- !query +DESC TABLE vv1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv1`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +alter table tt5 add column cc int +-- !query analysis +AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt5`, [StructField(cc,IntegerType,true)] + + +-- !query +DESC TABLE vv1 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv1`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +create table tt7 (x int, /* xx int, */ y int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt7`, false + + +-- !query +create table tt8 (x int, z int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt8`, false + + +-- !query +create view vv2 as +select * from (values(1,2,3,4,5)) v(a,b,c,d,e) +union all +select * from tt7 full join tt8 using (x), tt8 tt8x +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`vv2`, select * from (values(1,2,3,4,5)) v(a,b,c,d,e) +union all +select * from tt7 full join tt8 using (x), tt8 tt8x, false, false, PersistedView, true + +- Union false, false + :- Project [a#x, b#x, c#x, d#x, e#x] + : +- SubqueryAlias v + : +- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x, col5#x AS e#x] + : +- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x] + +- Project [x#x, y#x, z#x, x#x, z#x] + +- Join Inner + :- Project [coalesce(x#x, x#x) AS x#x, y#x, z#x] + : +- Join FullOuter, (x#x = x#x) + : :- SubqueryAlias spark_catalog.testviewschm2.tt7 + : : +- Relation spark_catalog.testviewschm2.tt7[x#x,y#x] parquet + : +- SubqueryAlias spark_catalog.testviewschm2.tt8 + : +- Relation spark_catalog.testviewschm2.tt8[x#x,z#x] parquet + +- SubqueryAlias tt8x + +- SubqueryAlias spark_catalog.testviewschm2.tt8 + +- Relation spark_catalog.testviewschm2.tt8[x#x,z#x] parquet + + +-- !query +DESC TABLE vv2 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv2`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +create view vv3 as +select * from (values(1,2,3,4,5,6)) v(a,b,c,x,e,f) +union all +select * from + tt7 full join tt8 using (x), + tt7 tt7x full join tt8 tt8x using (x) +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`vv3`, select * from (values(1,2,3,4,5,6)) v(a,b,c,x,e,f) +union all +select * from + tt7 full join tt8 using (x), + tt7 tt7x full join tt8 tt8x using (x), false, false, PersistedView, true + +- Union false, false + :- Project [a#x, b#x, c#x, x#x, e#x, f#x] + : +- SubqueryAlias v + : +- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS x#x, col5#x AS e#x, col6#x AS f#x] + : +- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x, col6#x] + +- Project [x#x, y#x, z#x, x#x, y#x, z#x] + +- Project [coalesce(x#x, x#x) AS x#x, y#x, z#x, x#x, y#x, z#x] + +- Join FullOuter, (x#x = x#x) + :- Join Inner + : :- Project [coalesce(x#x, x#x) AS x#x, y#x, z#x] + : : +- Join FullOuter, (x#x = x#x) + : : :- SubqueryAlias spark_catalog.testviewschm2.tt7 + : : : +- Relation spark_catalog.testviewschm2.tt7[x#x,y#x] parquet + : : +- SubqueryAlias spark_catalog.testviewschm2.tt8 + : : +- Relation spark_catalog.testviewschm2.tt8[x#x,z#x] parquet + : +- SubqueryAlias tt7x + : +- SubqueryAlias spark_catalog.testviewschm2.tt7 + : +- Relation spark_catalog.testviewschm2.tt7[x#x,y#x] parquet + +- SubqueryAlias tt8x + +- SubqueryAlias spark_catalog.testviewschm2.tt8 + +- Relation spark_catalog.testviewschm2.tt8[x#x,z#x] parquet + + +-- !query +DESC TABLE vv3 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv3`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +create view vv4 as +select * from (values(1,2,3,4,5,6,7)) v(a,b,c,x,e,f,g) +union all +select * from + tt7 full join tt8 using (x), + tt7 tt7x full join tt8 tt8x using (x) full join tt8 tt8y using (x) +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`vv4`, select * from (values(1,2,3,4,5,6,7)) v(a,b,c,x,e,f,g) +union all +select * from + tt7 full join tt8 using (x), + tt7 tt7x full join tt8 tt8x using (x) full join tt8 tt8y using (x), false, false, PersistedView, true + +- Union false, false + :- Project [a#x, b#x, c#x, x#x, e#x, f#x, g#x] + : +- SubqueryAlias v + : +- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS x#x, col5#x AS e#x, col6#x AS f#x, col7#x AS g#x] + : +- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x, col6#x, col7#x] + +- Project [x#x, y#x, z#x, x#x, y#x, z#x, z#x] + +- Project [coalesce(x#x, x#x) AS x#x, y#x, z#x, x#x, y#x, z#x, z#x] + +- Join FullOuter, (x#x = x#x) + :- Project [coalesce(x#x, x#x) AS x#x, y#x, z#x, x#x, y#x, z#x] + : +- Join FullOuter, (x#x = x#x) + : :- Join Inner + : : :- Project [coalesce(x#x, x#x) AS x#x, y#x, z#x] + : : : +- Join FullOuter, (x#x = x#x) + : : : :- SubqueryAlias spark_catalog.testviewschm2.tt7 + : : : : +- Relation spark_catalog.testviewschm2.tt7[x#x,y#x] parquet + : : : +- SubqueryAlias spark_catalog.testviewschm2.tt8 + : : : +- Relation spark_catalog.testviewschm2.tt8[x#x,z#x] parquet + : : +- SubqueryAlias tt7x + : : +- SubqueryAlias spark_catalog.testviewschm2.tt7 + : : +- Relation spark_catalog.testviewschm2.tt7[x#x,y#x] parquet + : +- SubqueryAlias tt8x + : +- SubqueryAlias spark_catalog.testviewschm2.tt8 + : +- Relation spark_catalog.testviewschm2.tt8[x#x,z#x] parquet + +- SubqueryAlias tt8y + +- SubqueryAlias spark_catalog.testviewschm2.tt8 + +- Relation spark_catalog.testviewschm2.tt8[x#x,z#x] parquet + + +-- !query +DESC TABLE vv4 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv4`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +alter table tt7 add column zz int +-- !query analysis +AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt7`, [StructField(zz,IntegerType,true)] + + +-- !query +alter table tt7 add column z int +-- !query analysis +AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt7`, [StructField(z,IntegerType,true)] + + +-- !query +alter table tt8 add column z2 int +-- !query analysis +AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt8`, [StructField(z2,IntegerType,true)] + + +-- !query +DESC TABLE vv2 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv2`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE vv3 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv3`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE vv4 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv4`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +create table tt7a (x date, /* xx int, */ y int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt7a`, false + + +-- !query +create table tt8a (x timestamp, z int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt8a`, false + + +-- !query +create view vv2a as +select * from (values(now(),2,3,now(),5)) v(a,b,c,d,e) +union all +select * from tt7a left join tt8a using (x), tt8a tt8ax +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`vv2a`, select * from (values(now(),2,3,now(),5)) v(a,b,c,d,e) +union all +select * from tt7a left join tt8a using (x), tt8a tt8ax, false, false, PersistedView, true + +- Union false, false + :- Project [a#x, b#x, c#x, d#x, e#x] + : +- SubqueryAlias v + : +- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x, col5#x AS e#x] + : +- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x] + +- Project [cast(x#x as timestamp) AS x#x, y#x, z#x, x#x, z#x] + +- Project [x#x, y#x, z#x, x#x, z#x] + +- Join Inner + :- Project [x#x, y#x, z#x] + : +- Join LeftOuter, (cast(x#x as timestamp) = x#x) + : :- SubqueryAlias spark_catalog.testviewschm2.tt7a + : : +- Relation spark_catalog.testviewschm2.tt7a[x#x,y#x] parquet + : +- SubqueryAlias spark_catalog.testviewschm2.tt8a + : +- Relation spark_catalog.testviewschm2.tt8a[x#x,z#x] parquet + +- SubqueryAlias tt8ax + +- SubqueryAlias spark_catalog.testviewschm2.tt8a + +- Relation spark_catalog.testviewschm2.tt8a[x#x,z#x] parquet + + +-- !query +DESC TABLE vv4 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv4`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE vv2a +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv2a`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +create table tt9 (x int, xx int, y int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt9`, false + + +-- !query +create table tt10 (x int, z int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt10`, false + + +-- !query +create view vv5 as select x,y,z from tt9 join tt10 using(x) +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`vv5`, select x,y,z from tt9 join tt10 using(x), false, false, PersistedView, true + +- Project [x#x, y#x, z#x] + +- Project [x#x, xx#x, y#x, z#x] + +- Join Inner, (x#x = x#x) + :- SubqueryAlias spark_catalog.testviewschm2.tt9 + : +- Relation spark_catalog.testviewschm2.tt9[x#x,xx#x,y#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.tt10 + +- Relation spark_catalog.testviewschm2.tt10[x#x,z#x] parquet + + +-- !query +DESC TABLE vv5 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv5`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DESC TABLE vv5 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv5`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +create table tt11 (x int, y int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt11`, false + + +-- !query +create table tt12 (x int, z int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt12`, false + + +-- !query +create table tt13 (z int, q int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tt13`, false + + +-- !query +create view vv6 as select x,y,z,q from + (tt11 join tt12 using(x)) join tt13 using(z) +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`vv6`, select x,y,z,q from + (tt11 join tt12 using(x)) join tt13 using(z), false, false, PersistedView, true + +- Project [x#x, y#x, z#x, q#x] + +- Project [z#x, x#x, y#x, q#x] + +- Join Inner, (z#x = z#x) + :- Project [x#x, y#x, z#x] + : +- Join Inner, (x#x = x#x) + : :- SubqueryAlias spark_catalog.testviewschm2.tt11 + : : +- Relation spark_catalog.testviewschm2.tt11[x#x,y#x] parquet + : +- SubqueryAlias spark_catalog.testviewschm2.tt12 + : +- Relation spark_catalog.testviewschm2.tt12[x#x,z#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.tt13 + +- Relation spark_catalog.testviewschm2.tt13[z#x,q#x] parquet + + +-- !query +DESC TABLE vv6 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv6`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +alter table tt11 add column z int +-- !query analysis +AlterTableAddColumnsCommand `spark_catalog`.`testviewschm2`.`tt11`, [StructField(z,IntegerType,true)] + + +-- !query +DESC TABLE vv6 +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`vv6`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +CREATE TABLE int8_tbl (q1 int, q2 int) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`int8_tbl`, false + + +-- !query +create view tt18v as + select * from int8_tbl xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxy + union all + select * from int8_tbl xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxz +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`tt18v`, select * from int8_tbl xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxy + union all + select * from int8_tbl xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxz, false, false, PersistedView, true + +- Union false, false + :- Project [q1#x, q2#x] + : +- SubqueryAlias xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxy + : +- SubqueryAlias spark_catalog.testviewschm2.int8_tbl + : +- Relation spark_catalog.testviewschm2.int8_tbl[q1#x,q2#x] parquet + +- Project [q1#x, q2#x] + +- SubqueryAlias xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxz + +- SubqueryAlias spark_catalog.testviewschm2.int8_tbl + +- Relation spark_catalog.testviewschm2.int8_tbl[q1#x,q2#x] parquet + + +-- !query +DESC TABLE tt18v +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`tt18v`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +create view tt21v as +select * from tt5 natural inner join tt6 +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`tt21v`, select * from tt5 natural inner join tt6, false, false, PersistedView, true + +- Project [c#x, a#x, b#x, cc#x, d#x] + +- Project [c#x, a#x, b#x, cc#x, d#x] + +- Join Inner, (c#x = c#x) + :- SubqueryAlias spark_catalog.testviewschm2.tt5 + : +- Relation spark_catalog.testviewschm2.tt5[a#x,b#x,c#x,cc#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.tt6 + +- Relation spark_catalog.testviewschm2.tt6[c#x,d#x] parquet + + +-- !query +DESC TABLE tt21v +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`tt21v`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +create view tt22v as +select * from tt5 natural left join tt6 +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`tt22v`, select * from tt5 natural left join tt6, false, false, PersistedView, true + +- Project [c#x, a#x, b#x, cc#x, d#x] + +- Project [c#x, a#x, b#x, cc#x, d#x] + +- Join LeftOuter, (c#x = c#x) + :- SubqueryAlias spark_catalog.testviewschm2.tt5 + : +- Relation spark_catalog.testviewschm2.tt5[a#x,b#x,c#x,cc#x] parquet + +- SubqueryAlias spark_catalog.testviewschm2.tt6 + +- Relation spark_catalog.testviewschm2.tt6[c#x,d#x] parquet + + +-- !query +DESC TABLE tt22v +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`tt22v`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +create view tt23v (col_a, col_b) as +select q1 as other_name1, q2 as other_name2 from int8_tbl +union +select 42, 43 +-- !query analysis +CreateViewCommand `spark_catalog`.`testviewschm2`.`tt23v`, [(col_a,None), (col_b,None)], select q1 as other_name1, q2 as other_name2 from int8_tbl +union +select 42, 43, false, false, PersistedView, true + +- Distinct + +- Union false, false + :- Project [q1#x AS other_name1#x, q2#x AS other_name2#x] + : +- SubqueryAlias spark_catalog.testviewschm2.int8_tbl + : +- Relation spark_catalog.testviewschm2.int8_tbl[q1#x,q2#x] parquet + +- Project [42 AS 42#x, 43 AS 43#x] + +- OneRowRelation + + +-- !query +DESC TABLE tt23v +-- !query analysis +DescribeTableCommand `spark_catalog`.`testviewschm2`.`tt23v`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +DROP SCHEMA temp_view_test CASCADE +-- !query analysis +DropNamespace false, true ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [temp_view_test] + + +-- !query +DROP SCHEMA testviewschm2 CASCADE +-- !query analysis +DropNamespace false, true ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [testviewschm2] + + +-- !query +DROP VIEW temp_table +-- !query analysis +DropTempViewCommand temp_table + + +-- !query +DROP VIEW tt +-- !query analysis +DropTempViewCommand tt diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/date.sql.out new file mode 100644 index 0000000000000..3f451a4d365b8 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/date.sql.out @@ -0,0 +1,583 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE DATE_TBL (f1 date) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`DATE_TBL`, false + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1957-04-09')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1] ++- Project [cast(col1#x as date) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1957-06-13')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1] ++- Project [cast(col1#x as date) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1996-02-28')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1] ++- Project [cast(col1#x as date) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1996-02-29')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1] ++- Project [cast(col1#x as date) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1996-03-01')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1] ++- Project [cast(col1#x as date) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1996-03-02')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1] ++- Project [cast(col1#x as date) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1997-02-28')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1] ++- Project [cast(col1#x as date) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1997-03-01')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1] ++- Project [cast(col1#x as date) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1997-03-02')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1] ++- Project [cast(col1#x as date) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO DATE_TBL VALUES (date('2000-04-01')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1] ++- Project [cast(col1#x as date) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO DATE_TBL VALUES (date('2000-04-02')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1] ++- Project [cast(col1#x as date) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO DATE_TBL VALUES (date('2000-04-03')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1] ++- Project [cast(col1#x as date) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO DATE_TBL VALUES (date('2038-04-08')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1] ++- Project [cast(col1#x as date) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO DATE_TBL VALUES (date('2039-04-09')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1] ++- Project [cast(col1#x as date) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO DATE_TBL VALUES (date('2040-04-10')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1] ++- Project [cast(col1#x as date) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT f1 AS `Fifteen` FROM DATE_TBL +-- !query analysis +Project [f1#x AS Fifteen#x] ++- SubqueryAlias spark_catalog.default.date_tbl + +- Relation spark_catalog.default.date_tbl[f1#x] parquet + + +-- !query +SELECT f1 AS `Nine` FROM DATE_TBL WHERE f1 < '2000-01-01' +-- !query analysis +Project [f1#x AS Nine#x] ++- Filter (f1#x < cast(2000-01-01 as date)) + +- SubqueryAlias spark_catalog.default.date_tbl + +- Relation spark_catalog.default.date_tbl[f1#x] parquet + + +-- !query +SELECT f1 AS `Three` FROM DATE_TBL + WHERE f1 BETWEEN '2000-01-01' AND '2001-01-01' +-- !query analysis +Project [f1#x AS Three#x] ++- Filter ((f1#x >= cast(2000-01-01 as date)) AND (f1#x <= cast(2001-01-01 as date))) + +- SubqueryAlias spark_catalog.default.date_tbl + +- Relation spark_catalog.default.date_tbl[f1#x] parquet + + +-- !query +SELECT date '1999-01-08' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date '1999-01-18' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date '1999 Jan 08' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 Jan 08'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 Jan 08'" + } ] +} + + +-- !query +SELECT date '1999 08 Jan' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 08 Jan'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 08 Jan'" + } ] +} + + +-- !query +SELECT date '1999-01-08' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date '1999-08-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date '1999 01 08' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 01 08'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 01 08'" + } ] +} + + +-- !query +SELECT date '1999 08 01' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 08 01'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 08 01'" + } ] +} + + +-- !query +SELECT date '1999-01-08' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date '1999 Jan 08' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 Jan 08'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 Jan 08'" + } ] +} + + +-- !query +SELECT date '1999 08 Jan' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 08 Jan'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 08 Jan'" + } ] +} + + +-- !query +SELECT date '1999-01-08' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date '1999-08-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date '1999 01 08' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 01 08'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 01 08'" + } ] +} + + +-- !query +SELECT date '1999 08 01' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 08 01'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 08 01'" + } ] +} + + +-- !query +SELECT date '1999-01-08' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date '1999-01-18' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date '1999 Jan 08' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 Jan 08'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 Jan 08'" + } ] +} + + +-- !query +SELECT date '1999 08 Jan' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 08 Jan'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 08 Jan'" + } ] +} + + +-- !query +SELECT date '1999-01-08' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date '1999-08-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date '1999 01 08' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 01 08'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 01 08'" + } ] +} + + +-- !query +SELECT date '1999 08 01' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 08 01'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 08 01'" + } ] +} + + +-- !query +SELECT date '4714-11-24 BC' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date '4714-11-23 BC' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date '5874897-12-31' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date '5874898-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT f1 - date 'epoch' AS `Days From Epoch` FROM DATE_TBL +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date 'yesterday' - date 'today' AS `One day` +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date 'today' - date 'tomorrow' AS `One day` +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date 'yesterday' - date 'tomorrow' AS `Two days` +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date 'tomorrow' - date 'today' AS `One day` +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date 'today' - date 'yesterday' AS `One day` +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT date 'tomorrow' - date 'yesterday' AS `Two days` +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select make_date(2013, 7, 15) +-- !query analysis +Project [make_date(2013, 7, 15, true) AS make_date(2013, 7, 15)#x] ++- OneRowRelation + + +-- !query +select make_date(-44, 3, 15) +-- !query analysis +Project [make_date(-44, 3, 15, true) AS make_date(-44, 3, 15)#x] ++- OneRowRelation + + +-- !query +select make_date(2013, 2, 30) +-- !query analysis +Project [make_date(2013, 2, 30, true) AS make_date(2013, 2, 30)#x] ++- OneRowRelation + + +-- !query +select make_date(2013, 13, 1) +-- !query analysis +Project [make_date(2013, 13, 1, true) AS make_date(2013, 13, 1)#x] ++- OneRowRelation + + +-- !query +select make_date(2013, 11, -1) +-- !query analysis +Project [make_date(2013, 11, -1, true) AS make_date(2013, 11, -1)#x] ++- OneRowRelation + + +-- !query +DROP TABLE DATE_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`DATE_TBL`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float4.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float4.sql.out new file mode 100644 index 0000000000000..22dd493eb91b6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float4.sql.out @@ -0,0 +1,339 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE FLOAT4_TBL (f1 float) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`FLOAT4_TBL`, false + + +-- !query +INSERT INTO FLOAT4_TBL VALUES (float(' 0.0')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float4_tbl], Append, `spark_catalog`.`default`.`float4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float4_tbl), [f1] ++- Project [cast(col1#x as float) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO FLOAT4_TBL VALUES (float('1004.30 ')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float4_tbl], Append, `spark_catalog`.`default`.`float4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float4_tbl), [f1] ++- Project [cast(col1#x as float) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO FLOAT4_TBL VALUES (float(' -34.84 ')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float4_tbl], Append, `spark_catalog`.`default`.`float4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float4_tbl), [f1] ++- Project [cast(col1#x as float) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e+20')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float4_tbl], Append, `spark_catalog`.`default`.`float4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float4_tbl), [f1] ++- Project [cast(col1#x as float) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e-20')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float4_tbl], Append, `spark_catalog`.`default`.`float4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float4_tbl), [f1] ++- Project [cast(col1#x as float) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT float('NaN') +-- !query analysis +Project [cast(NaN as float) AS NaN#x] ++- OneRowRelation + + +-- !query +SELECT float('nan') +-- !query analysis +Project [cast(nan as float) AS nan#x] ++- OneRowRelation + + +-- !query +SELECT float(' NAN ') +-- !query analysis +Project [cast( NAN as float) AS NAN #x] ++- OneRowRelation + + +-- !query +SELECT float('infinity') +-- !query analysis +Project [cast(infinity as float) AS infinity#x] ++- OneRowRelation + + +-- !query +SELECT float(' -INFINiTY ') +-- !query analysis +Project [cast( -INFINiTY as float) AS -INFINiTY #x] ++- OneRowRelation + + +-- !query +SELECT float('N A N') +-- !query analysis +Project [cast(N A N as float) AS N A N#x] ++- OneRowRelation + + +-- !query +SELECT float('NaN x') +-- !query analysis +Project [cast(NaN x as float) AS NaN x#x] ++- OneRowRelation + + +-- !query +SELECT float(' INFINITY x') +-- !query analysis +Project [cast( INFINITY x as float) AS INFINITY x#x] ++- OneRowRelation + + +-- !query +SELECT float('Infinity') + 100.0 +-- !query analysis +Project [(cast(cast(Infinity as float) as double) + cast(100.0 as double)) AS (Infinity + 100.0)#x] ++- OneRowRelation + + +-- !query +SELECT float('Infinity') / float('Infinity') +-- !query analysis +Project [(cast(cast(Infinity as float) as double) / cast(cast(Infinity as float) as double)) AS (Infinity / Infinity)#x] ++- OneRowRelation + + +-- !query +SELECT float('nan') / float('nan') +-- !query analysis +Project [(cast(cast(nan as float) as double) / cast(cast(nan as float) as double)) AS (nan / nan)#x] ++- OneRowRelation + + +-- !query +SELECT float(decimal('nan')) +-- !query analysis +Project [cast(cast(nan as decimal(10,0)) as float) AS nan#x] ++- OneRowRelation + + +-- !query +SELECT '' AS five, * FROM FLOAT4_TBL +-- !query analysis +Project [ AS five#x, f1#x] ++- SubqueryAlias spark_catalog.default.float4_tbl + +- Relation spark_catalog.default.float4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE f.f1 <> '1004.3' +-- !query analysis +Project [ AS four#x, f1#x] ++- Filter NOT (cast(f1#x as double) = cast(1004.3 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float4_tbl + +- Relation spark_catalog.default.float4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS one, f.* FROM FLOAT4_TBL f WHERE f.f1 = '1004.3' +-- !query analysis +Project [ AS one#x, f1#x] ++- Filter (cast(f1#x as double) = cast(1004.3 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float4_tbl + +- Relation spark_catalog.default.float4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, f.* FROM FLOAT4_TBL f WHERE '1004.3' > f.f1 +-- !query analysis +Project [ AS three#x, f1#x] ++- Filter (cast(1004.3 as double) > cast(f1#x as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float4_tbl + +- Relation spark_catalog.default.float4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, f.* FROM FLOAT4_TBL f WHERE f.f1 < '1004.3' +-- !query analysis +Project [ AS three#x, f1#x] ++- Filter (cast(f1#x as double) < cast(1004.3 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float4_tbl + +- Relation spark_catalog.default.float4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE '1004.3' >= f.f1 +-- !query analysis +Project [ AS four#x, f1#x] ++- Filter (cast(1004.3 as double) >= cast(f1#x as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float4_tbl + +- Relation spark_catalog.default.float4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE f.f1 <= '1004.3' +-- !query analysis +Project [ AS four#x, f1#x] ++- Filter (cast(f1#x as double) <= cast(1004.3 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float4_tbl + +- Relation spark_catalog.default.float4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, f.f1, f.f1 * '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query analysis +Project [ AS three#x, f1#x, (cast(f1#x as double) * cast(-10 as double)) AS x#x] ++- Filter (cast(f1#x as double) > cast(0.0 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float4_tbl + +- Relation spark_catalog.default.float4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, f.f1, f.f1 + '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query analysis +Project [ AS three#x, f1#x, (cast(f1#x as double) + cast(-10 as double)) AS x#x] ++- Filter (cast(f1#x as double) > cast(0.0 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float4_tbl + +- Relation spark_catalog.default.float4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, f.f1, f.f1 / '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query analysis +Project [ AS three#x, f1#x, (cast(f1#x as double) / cast(-10 as double)) AS x#x] ++- Filter (cast(f1#x as double) > cast(0.0 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float4_tbl + +- Relation spark_catalog.default.float4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, f.f1, f.f1 - '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query analysis +Project [ AS three#x, f1#x, (cast(f1#x as double) - cast(-10 as double)) AS x#x] ++- Filter (cast(f1#x as double) > cast(0.0 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float4_tbl + +- Relation spark_catalog.default.float4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, * FROM FLOAT4_TBL +-- !query analysis +Project [ AS five#x, f1#x] ++- SubqueryAlias spark_catalog.default.float4_tbl + +- Relation spark_catalog.default.float4_tbl[f1#x] parquet + + +-- !query +SELECT smallint(float('32767.4')) +-- !query analysis +Project [cast(cast(32767.4 as float) as smallint) AS 32767.4#x] ++- OneRowRelation + + +-- !query +SELECT smallint(float('32767.6')) +-- !query analysis +Project [cast(cast(32767.6 as float) as smallint) AS 32767.6#x] ++- OneRowRelation + + +-- !query +SELECT smallint(float('-32768.4')) +-- !query analysis +Project [cast(cast(-32768.4 as float) as smallint) AS -32768.4#x] ++- OneRowRelation + + +-- !query +SELECT smallint(float('-32768.6')) +-- !query analysis +Project [cast(cast(-32768.6 as float) as smallint) AS -32768.6#x] ++- OneRowRelation + + +-- !query +SELECT int(float('2147483520')) +-- !query analysis +Project [cast(cast(2147483520 as float) as int) AS 2147483520#x] ++- OneRowRelation + + +-- !query +SELECT int(float('2147483647')) +-- !query analysis +Project [cast(cast(2147483647 as float) as int) AS 2147483647#x] ++- OneRowRelation + + +-- !query +SELECT int(float('-2147483648.5')) +-- !query analysis +Project [cast(cast(-2147483648.5 as float) as int) AS -2147483648.5#x] ++- OneRowRelation + + +-- !query +SELECT int(float('-2147483900')) +-- !query analysis +Project [cast(cast(-2147483900 as float) as int) AS -2147483900#x] ++- OneRowRelation + + +-- !query +SELECT bigint(float('9223369837831520256')) +-- !query analysis +Project [cast(cast(9223369837831520256 as float) as bigint) AS 9223369837831520256#xL] ++- OneRowRelation + + +-- !query +SELECT bigint(float('9223372036854775807')) +-- !query analysis +Project [cast(cast(9223372036854775807 as float) as bigint) AS 9223372036854775807#xL] ++- OneRowRelation + + +-- !query +SELECT bigint(float('-9223372036854775808.5')) +-- !query analysis +Project [cast(cast(-9223372036854775808.5 as float) as bigint) AS -9223372036854775808.5#xL] ++- OneRowRelation + + +-- !query +SELECT bigint(float('-9223380000000000000')) +-- !query analysis +Project [cast(cast(-9223380000000000000 as float) as bigint) AS -9223380000000000000#xL] ++- OneRowRelation + + +-- !query +DROP TABLE FLOAT4_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`FLOAT4_TBL`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float8.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float8.sql.out new file mode 100644 index 0000000000000..b8861f07201e2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float8.sql.out @@ -0,0 +1,746 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE FLOAT8_TBL(f1 double) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`FLOAT8_TBL`, false + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double(' 0.0 ')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1] ++- Project [cast(col1#x as double) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('1004.30 ')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1] ++- Project [cast(col1#x as double) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double(' -34.84')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1] ++- Project [cast(col1#x as double) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e+200')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1] ++- Project [cast(col1#x as double) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e-200')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1] ++- Project [cast(col1#x as double) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT double('10e400') +-- !query analysis +Project [cast(10e400 as double) AS 10e400#x] ++- OneRowRelation + + +-- !query +SELECT double('-10e400') +-- !query analysis +Project [cast(-10e400 as double) AS -10e400#x] ++- OneRowRelation + + +-- !query +SELECT double('10e-400') +-- !query analysis +Project [cast(10e-400 as double) AS 10e-400#x] ++- OneRowRelation + + +-- !query +SELECT double('-10e-400') +-- !query analysis +Project [cast(-10e-400 as double) AS -10e-400#x] ++- OneRowRelation + + +-- !query +SELECT double('NaN') +-- !query analysis +Project [cast(NaN as double) AS NaN#x] ++- OneRowRelation + + +-- !query +SELECT double('nan') +-- !query analysis +Project [cast(nan as double) AS nan#x] ++- OneRowRelation + + +-- !query +SELECT double(' NAN ') +-- !query analysis +Project [cast( NAN as double) AS NAN #x] ++- OneRowRelation + + +-- !query +SELECT double('infinity') +-- !query analysis +Project [cast(infinity as double) AS infinity#x] ++- OneRowRelation + + +-- !query +SELECT double(' -INFINiTY ') +-- !query analysis +Project [cast( -INFINiTY as double) AS -INFINiTY #x] ++- OneRowRelation + + +-- !query +SELECT double('N A N') +-- !query analysis +Project [cast(N A N as double) AS N A N#x] ++- OneRowRelation + + +-- !query +SELECT double('NaN x') +-- !query analysis +Project [cast(NaN x as double) AS NaN x#x] ++- OneRowRelation + + +-- !query +SELECT double(' INFINITY x') +-- !query analysis +Project [cast( INFINITY x as double) AS INFINITY x#x] ++- OneRowRelation + + +-- !query +SELECT double('Infinity') + 100.0 +-- !query analysis +Project [(cast(Infinity as double) + cast(100.0 as double)) AS (Infinity + 100.0)#x] ++- OneRowRelation + + +-- !query +SELECT double('Infinity') / double('Infinity') +-- !query analysis +Project [(cast(Infinity as double) / cast(Infinity as double)) AS (Infinity / Infinity)#x] ++- OneRowRelation + + +-- !query +SELECT double('NaN') / double('NaN') +-- !query analysis +Project [(cast(NaN as double) / cast(NaN as double)) AS (NaN / NaN)#x] ++- OneRowRelation + + +-- !query +SELECT double(decimal('nan')) +-- !query analysis +Project [cast(cast(nan as decimal(10,0)) as double) AS nan#x] ++- OneRowRelation + + +-- !query +SELECT '' AS five, * FROM FLOAT8_TBL +-- !query analysis +Project [ AS five#x, f1#x] ++- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <> '1004.3' +-- !query analysis +Project [ AS four#x, f1#x] ++- Filter NOT (f1#x = cast(1004.3 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT '' AS one, f.* FROM FLOAT8_TBL f WHERE f.f1 = '1004.3' +-- !query analysis +Project [ AS one#x, f1#x] ++- Filter (f1#x = cast(1004.3 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE '1004.3' > f.f1 +-- !query analysis +Project [ AS three#x, f1#x] ++- Filter (cast(1004.3 as double) > f1#x) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE f.f1 < '1004.3' +-- !query analysis +Project [ AS three#x, f1#x] ++- Filter (f1#x < cast(1004.3 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE '1004.3' >= f.f1 +-- !query analysis +Project [ AS four#x, f1#x] ++- Filter (cast(1004.3 as double) >= f1#x) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <= '1004.3' +-- !query analysis +Project [ AS four#x, f1#x] ++- Filter (f1#x <= cast(1004.3 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, f.f1, f.f1 * '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query analysis +Project [ AS three#x, f1#x, (f1#x * cast(-10 as double)) AS x#x] ++- Filter (f1#x > cast(0.0 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, f.f1, f.f1 + '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query analysis +Project [ AS three#x, f1#x, (f1#x + cast(-10 as double)) AS x#x] ++- Filter (f1#x > cast(0.0 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, f.f1, f.f1 / '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query analysis +Project [ AS three#x, f1#x, (f1#x / cast(-10 as double)) AS x#x] ++- Filter (f1#x > cast(0.0 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, f.f1, f.f1 - '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query analysis +Project [ AS three#x, f1#x, (f1#x - cast(-10 as double)) AS x#x] ++- Filter (f1#x > cast(0.0 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, f.f1, round(f.f1) AS round_f1 + FROM FLOAT8_TBL f +-- !query analysis +Project [ AS five#x, f1#x, round(f1#x, 0) AS round_f1#x] ++- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +select ceil(f1) as ceil_f1 from float8_tbl f +-- !query analysis +Project [CEIL(f1#x) AS ceil_f1#xL] ++- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +select ceiling(f1) as ceiling_f1 from float8_tbl f +-- !query analysis +Project [ceiling(f1#x) AS ceiling_f1#xL] ++- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +select floor(f1) as floor_f1 from float8_tbl f +-- !query analysis +Project [FLOOR(f1#x) AS floor_f1#xL] ++- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +select sign(f1) as sign_f1 from float8_tbl f +-- !query analysis +Project [sign(f1#x) AS sign_f1#x] ++- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT sqrt(double('64')) AS eight +-- !query analysis +Project [SQRT(cast(64 as double)) AS eight#x] ++- OneRowRelation + + +-- !query +SELECT power(double('144'), double('0.5')) +-- !query analysis +Project [POWER(cast(144 as double), cast(0.5 as double)) AS POWER(144, 0.5)#x] ++- OneRowRelation + + +-- !query +SELECT power(double('NaN'), double('0.5')) +-- !query analysis +Project [POWER(cast(NaN as double), cast(0.5 as double)) AS POWER(NaN, 0.5)#x] ++- OneRowRelation + + +-- !query +SELECT power(double('144'), double('NaN')) +-- !query analysis +Project [POWER(cast(144 as double), cast(NaN as double)) AS POWER(144, NaN)#x] ++- OneRowRelation + + +-- !query +SELECT power(double('NaN'), double('NaN')) +-- !query analysis +Project [POWER(cast(NaN as double), cast(NaN as double)) AS POWER(NaN, NaN)#x] ++- OneRowRelation + + +-- !query +SELECT power(double('-1'), double('NaN')) +-- !query analysis +Project [POWER(cast(-1 as double), cast(NaN as double)) AS POWER(-1, NaN)#x] ++- OneRowRelation + + +-- !query +SELECT power(double('1'), double('NaN')) +-- !query analysis +Project [POWER(cast(1 as double), cast(NaN as double)) AS POWER(1, NaN)#x] ++- OneRowRelation + + +-- !query +SELECT power(double('NaN'), double('0')) +-- !query analysis +Project [POWER(cast(NaN as double), cast(0 as double)) AS POWER(NaN, 0)#x] ++- OneRowRelation + + +-- !query +SELECT '' AS three, f.f1, exp(ln(f.f1)) AS exp_ln_f1 + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query analysis +Project [ AS three#x, f1#x, EXP(ln(f1#x)) AS exp_ln_f1#x] ++- Filter (f1#x > cast(0.0 as double)) + +- SubqueryAlias f + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, * FROM FLOAT8_TBL +-- !query analysis +Project [ AS five#x, f1#x] ++- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +CREATE TEMPORARY VIEW UPDATED_FLOAT8_TBL as +SELECT + CASE WHEN FLOAT8_TBL.f1 > '0.0' THEN FLOAT8_TBL.f1 * '-1' ELSE FLOAT8_TBL.f1 END AS f1 +FROM FLOAT8_TBL +-- !query analysis +CreateViewCommand `UPDATED_FLOAT8_TBL`, SELECT + CASE WHEN FLOAT8_TBL.f1 > '0.0' THEN FLOAT8_TBL.f1 * '-1' ELSE FLOAT8_TBL.f1 END AS f1 +FROM FLOAT8_TBL, false, false, LocalTempView, true + +- Project [CASE WHEN (f1#x > cast(0.0 as double)) THEN (f1#x * cast(-1 as double)) ELSE f1#x END AS f1#x] + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT '' AS bad, f.f1 * '1e200' from UPDATED_FLOAT8_TBL f +-- !query analysis +Project [ AS bad#x, (f1#x * cast(1e200 as double)) AS (f1 * 1e200)#x] ++- SubqueryAlias f + +- SubqueryAlias updated_float8_tbl + +- View (`UPDATED_FLOAT8_TBL`, [f1#x]) + +- Project [cast(f1#x as double) AS f1#x] + +- Project [CASE WHEN (f1#x > cast(0.0 as double)) THEN (f1#x * cast(-1 as double)) ELSE f1#x END AS f1#x] + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, * FROM UPDATED_FLOAT8_TBL +-- !query analysis +Project [ AS five#x, f1#x] ++- SubqueryAlias updated_float8_tbl + +- View (`UPDATED_FLOAT8_TBL`, [f1#x]) + +- Project [cast(f1#x as double) AS f1#x] + +- Project [CASE WHEN (f1#x > cast(0.0 as double)) THEN (f1#x * cast(-1 as double)) ELSE f1#x END AS f1#x] + +- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT sinh(double('1')) +-- !query analysis +Project [SINH(cast(1 as double)) AS SINH(1)#x] ++- OneRowRelation + + +-- !query +SELECT cosh(double('1')) +-- !query analysis +Project [COSH(cast(1 as double)) AS COSH(1)#x] ++- OneRowRelation + + +-- !query +SELECT tanh(double('1')) +-- !query analysis +Project [TANH(cast(1 as double)) AS TANH(1)#x] ++- OneRowRelation + + +-- !query +SELECT asinh(double('1')) +-- !query analysis +Project [ASINH(cast(1 as double)) AS ASINH(1)#x] ++- OneRowRelation + + +-- !query +SELECT acosh(double('2')) +-- !query analysis +Project [ACOSH(cast(2 as double)) AS ACOSH(2)#x] ++- OneRowRelation + + +-- !query +SELECT atanh(double('0.5')) +-- !query analysis +Project [ATANH(cast(0.5 as double)) AS ATANH(0.5)#x] ++- OneRowRelation + + +-- !query +SELECT sinh(double('Infinity')) +-- !query analysis +Project [SINH(cast(Infinity as double)) AS SINH(Infinity)#x] ++- OneRowRelation + + +-- !query +SELECT sinh(double('-Infinity')) +-- !query analysis +Project [SINH(cast(-Infinity as double)) AS SINH(-Infinity)#x] ++- OneRowRelation + + +-- !query +SELECT sinh(double('NaN')) +-- !query analysis +Project [SINH(cast(NaN as double)) AS SINH(NaN)#x] ++- OneRowRelation + + +-- !query +SELECT cosh(double('Infinity')) +-- !query analysis +Project [COSH(cast(Infinity as double)) AS COSH(Infinity)#x] ++- OneRowRelation + + +-- !query +SELECT cosh(double('-Infinity')) +-- !query analysis +Project [COSH(cast(-Infinity as double)) AS COSH(-Infinity)#x] ++- OneRowRelation + + +-- !query +SELECT cosh(double('NaN')) +-- !query analysis +Project [COSH(cast(NaN as double)) AS COSH(NaN)#x] ++- OneRowRelation + + +-- !query +SELECT tanh(double('Infinity')) +-- !query analysis +Project [TANH(cast(Infinity as double)) AS TANH(Infinity)#x] ++- OneRowRelation + + +-- !query +SELECT tanh(double('-Infinity')) +-- !query analysis +Project [TANH(cast(-Infinity as double)) AS TANH(-Infinity)#x] ++- OneRowRelation + + +-- !query +SELECT tanh(double('NaN')) +-- !query analysis +Project [TANH(cast(NaN as double)) AS TANH(NaN)#x] ++- OneRowRelation + + +-- !query +SELECT asinh(double('Infinity')) +-- !query analysis +Project [ASINH(cast(Infinity as double)) AS ASINH(Infinity)#x] ++- OneRowRelation + + +-- !query +SELECT asinh(double('-Infinity')) +-- !query analysis +Project [ASINH(cast(-Infinity as double)) AS ASINH(-Infinity)#x] ++- OneRowRelation + + +-- !query +SELECT asinh(double('NaN')) +-- !query analysis +Project [ASINH(cast(NaN as double)) AS ASINH(NaN)#x] ++- OneRowRelation + + +-- !query +SELECT acosh(double('Infinity')) +-- !query analysis +Project [ACOSH(cast(Infinity as double)) AS ACOSH(Infinity)#x] ++- OneRowRelation + + +-- !query +SELECT acosh(double('-Infinity')) +-- !query analysis +Project [ACOSH(cast(-Infinity as double)) AS ACOSH(-Infinity)#x] ++- OneRowRelation + + +-- !query +SELECT acosh(double('NaN')) +-- !query analysis +Project [ACOSH(cast(NaN as double)) AS ACOSH(NaN)#x] ++- OneRowRelation + + +-- !query +SELECT atanh(double('Infinity')) +-- !query analysis +Project [ATANH(cast(Infinity as double)) AS ATANH(Infinity)#x] ++- OneRowRelation + + +-- !query +SELECT atanh(double('-Infinity')) +-- !query analysis +Project [ATANH(cast(-Infinity as double)) AS ATANH(-Infinity)#x] ++- OneRowRelation + + +-- !query +SELECT atanh(double('NaN')) +-- !query analysis +Project [ATANH(cast(NaN as double)) AS ATANH(NaN)#x] ++- OneRowRelation + + +-- !query +TRUNCATE TABLE FLOAT8_TBL +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`float8_tbl` + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('0.0')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1] ++- Project [cast(col1#x as double) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('-34.84')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1] ++- Project [cast(col1#x as double) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('-1004.30')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1] ++- Project [cast(col1#x as double) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e+200')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1] ++- Project [cast(col1#x as double) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e-200')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1] ++- Project [cast(col1#x as double) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT '' AS five, * FROM FLOAT8_TBL +-- !query analysis +Project [ AS five#x, f1#x] ++- SubqueryAlias spark_catalog.default.float8_tbl + +- Relation spark_catalog.default.float8_tbl[f1#x] parquet + + +-- !query +SELECT smallint(double('32767.4')) +-- !query analysis +Project [cast(cast(32767.4 as double) as smallint) AS 32767.4#x] ++- OneRowRelation + + +-- !query +SELECT smallint(double('32767.6')) +-- !query analysis +Project [cast(cast(32767.6 as double) as smallint) AS 32767.6#x] ++- OneRowRelation + + +-- !query +SELECT smallint(double('-32768.4')) +-- !query analysis +Project [cast(cast(-32768.4 as double) as smallint) AS -32768.4#x] ++- OneRowRelation + + +-- !query +SELECT smallint(double('-32768.6')) +-- !query analysis +Project [cast(cast(-32768.6 as double) as smallint) AS -32768.6#x] ++- OneRowRelation + + +-- !query +SELECT int(double('2147483647.4')) +-- !query analysis +Project [cast(cast(2147483647.4 as double) as int) AS 2147483647.4#x] ++- OneRowRelation + + +-- !query +SELECT int(double('2147483647.6')) +-- !query analysis +Project [cast(cast(2147483647.6 as double) as int) AS 2147483647.6#x] ++- OneRowRelation + + +-- !query +SELECT int(double('-2147483648.4')) +-- !query analysis +Project [cast(cast(-2147483648.4 as double) as int) AS -2147483648.4#x] ++- OneRowRelation + + +-- !query +SELECT int(double('-2147483648.6')) +-- !query analysis +Project [cast(cast(-2147483648.6 as double) as int) AS -2147483648.6#x] ++- OneRowRelation + + +-- !query +SELECT bigint(double('9223372036854773760')) +-- !query analysis +Project [cast(cast(9223372036854773760 as double) as bigint) AS 9223372036854773760#xL] ++- OneRowRelation + + +-- !query +SELECT bigint(double('9223372036854775807')) +-- !query analysis +Project [cast(cast(9223372036854775807 as double) as bigint) AS 9223372036854775807#xL] ++- OneRowRelation + + +-- !query +SELECT bigint(double('-9223372036854775808.5')) +-- !query analysis +Project [cast(cast(-9223372036854775808.5 as double) as bigint) AS -9223372036854775808.5#xL] ++- OneRowRelation + + +-- !query +SELECT bigint(double('-9223372036854780000')) +-- !query analysis +Project [cast(cast(-9223372036854780000 as double) as bigint) AS -9223372036854780000#xL] ++- OneRowRelation + + +-- !query +DROP TABLE FLOAT8_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`FLOAT8_TBL`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/groupingsets.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/groupingsets.sql.out new file mode 100644 index 0000000000000..f7fd784d9f965 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/groupingsets.sql.out @@ -0,0 +1,684 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temp view gstest1(a,b,v) + as values (1,1,10),(1,1,11),(1,2,12),(1,2,13),(1,3,14), + (2,3,15), + (3,3,16),(3,4,17), + (4,1,18),(4,1,19) +-- !query analysis +CreateViewCommand `gstest1`, [(a,None), (b,None), (v,None)], values (1,1,10),(1,1,11),(1,2,12),(1,2,13),(1,3,14), + (2,3,15), + (3,3,16),(3,4,17), + (4,1,18),(4,1,19), false, false, LocalTempView, true + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +create table gstest2 (a integer, b integer, c integer, d integer, + e integer, f integer, g integer, h integer) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`gstest2`, false + + +-- !query +insert into gstest2 values + (1, 1, 1, 1, 1, 1, 1, 1), + (1, 1, 1, 1, 1, 1, 1, 2), + (1, 1, 1, 1, 1, 1, 2, 2), + (1, 1, 1, 1, 1, 2, 2, 2), + (1, 1, 1, 1, 2, 2, 2, 2), + (1, 1, 1, 2, 2, 2, 2, 2), + (1, 1, 2, 2, 2, 2, 2, 2), + (1, 2, 2, 2, 2, 2, 2, 2), + (2, 2, 2, 2, 2, 2, 2, 2) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/gstest2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/gstest2], Append, `spark_catalog`.`default`.`gstest2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/gstest2), [a, b, c, d, e, f, g, h] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x, cast(col4#x as int) AS d#x, cast(col5#x as int) AS e#x, cast(col6#x as int) AS f#x, cast(col7#x as int) AS g#x, cast(col8#x as int) AS h#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x, col6#x, col7#x, col8#x] + + +-- !query +create table gstest3 (a integer, b integer, c integer, d integer) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`gstest3`, false + + +-- !query +insert into gstest3 values + (1, 1, 1, 1), + (2, 2, 2, 2) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/gstest3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/gstest3], Append, `spark_catalog`.`default`.`gstest3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/gstest3), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x, cast(col4#x as int) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +create table gstest4(id integer, v integer, + unhashable_col /* bit(4) */ byte, unsortable_col /* xid */ integer) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`gstest4`, false + + +-- !query +insert into gstest4 +values (1,1,tinyint('0'),1), (2,2,tinyint('1'),1), + (3,4,tinyint('2'),2), (4,8,tinyint('3'),2), + (5,16,tinyint('0'),2), (6,32,tinyint('1'),2), + (7,64,tinyint('2'),1), (8,128,tinyint('3'),1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/gstest4, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/gstest4], Append, `spark_catalog`.`default`.`gstest4`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/gstest4), [id, v, unhashable_col, unsortable_col] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS v#x, cast(col3#x as tinyint) AS unhashable_col#x, cast(col4#x as int) AS unsortable_col#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +create table gstest_empty (a integer, b integer, v integer) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`gstest_empty`, false + + +-- !query +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by rollup (a,b) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] ++- Expand [[a#x, b#x, v#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, null, 1], [a#x, b#x, v#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias gstest1 + +- View (`gstest1`, [a#x,b#x,v#x]) + +- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS v#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by rollup (a,b) order by a,b +-- !query analysis +Sort [a#x ASC NULLS FIRST, b#x ASC NULLS FIRST], true ++- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] + +- Expand [[a#x, b#x, v#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, null, 1], [a#x, b#x, v#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias gstest1 + +- View (`gstest1`, [a#x,b#x,v#x]) + +- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS v#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by rollup (a,b) order by b desc, a +-- !query analysis +Sort [b#x DESC NULLS LAST, a#x ASC NULLS FIRST], true ++- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] + +- Expand [[a#x, b#x, v#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, null, 1], [a#x, b#x, v#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias gstest1 + +- View (`gstest1`, [a#x,b#x,v#x]) + +- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS v#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by rollup (a,b) order by coalesce(a,0)+coalesce(b,0), a +-- !query analysis +Sort [(coalesce(a#x, 0) + coalesce(b#x, 0)) ASC NULLS FIRST, a#x ASC NULLS FIRST], true ++- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] + +- Expand [[a#x, b#x, v#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, null, 1], [a#x, b#x, v#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias gstest1 + +- View (`gstest1`, [a#x,b#x,v#x]) + +- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS v#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +select a, b, sum(c), sum(sum(c)) over (order by a,b) as rsum + from gstest2 group by rollup (a,b) order by rsum, a, b +-- !query analysis +Sort [rsum#xL ASC NULLS FIRST, a#x ASC NULLS FIRST, b#x ASC NULLS FIRST], true ++- Project [a#x, b#x, sum(c)#xL, rsum#xL] + +- Project [a#x, b#x, sum(c)#xL, _w0#xL, rsum#xL, rsum#xL] + +- Window [sum(_w0#xL) windowspecdefinition(a#x ASC NULLS FIRST, b#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS rsum#xL], [a#x ASC NULLS FIRST, b#x ASC NULLS FIRST] + +- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, sum(c#x) AS sum(c)#xL, sum(c#x) AS _w0#xL] + +- Expand [[a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, 0], [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, null, 1], [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, null, null, 3]], [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias spark_catalog.default.gstest2 + +- Relation spark_catalog.default.gstest2[a#x,b#x,c#x,d#x,e#x,f#x,g#x,h#x] parquet + + +-- !query +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),a) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, v#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, null, 1]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias spark_catalog.default.gstest_empty + +- Relation spark_catalog.default.gstest_empty[a#x,b#x,v#x] parquet + + +-- !query +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),()) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, v#x, a#x, b#x, 0], [a#x, b#x, v#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias spark_catalog.default.gstest_empty + +- Relation spark_catalog.default.gstest_empty[a#x,b#x,v#x] parquet + + +-- !query +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),(),(),()) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, v#x, a#x, b#x, 0, 0], [a#x, b#x, v#x, null, null, 3, 1], [a#x, b#x, v#x, null, null, 3, 2], [a#x, b#x, v#x, null, null, 3, 3]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias spark_catalog.default.gstest_empty + +- Relation spark_catalog.default.gstest_empty[a#x,b#x,v#x] parquet + + +-- !query +select sum(v), count(*) from gstest_empty group by grouping sets ((),(),()) +-- !query analysis +Aggregate [spark_grouping_id#xL, _gen_grouping_pos#x], [sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, v#x, 0, 0], [a#x, b#x, v#x, 0, 1], [a#x, b#x, v#x, 0, 2]], [a#x, b#x, v#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, v#x] + +- SubqueryAlias spark_catalog.default.gstest_empty + +- Relation spark_catalog.default.gstest_empty[a#x,b#x,v#x] parquet + + +-- !query +select t1.a, t2.b, sum(t1.v), count(*) from gstest_empty t1, gstest_empty t2 + group by grouping sets ((t1.a,t2.b),()) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, v#x, a#x, b#x, v#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, b#x, v#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, v#x, a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.gstest_empty + : +- Relation spark_catalog.default.gstest_empty[a#x,b#x,v#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.gstest_empty + +- Relation spark_catalog.default.gstest_empty[a#x,b#x,v#x] parquet + + +-- !query +select t1.a, t2.b, grouping(t1.a), grouping(t2.b), sum(t1.v), max(t2.a) + from gstest1 t1, gstest2 t2 + group by grouping sets ((t1.a, t2.b), ()) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, max(a#x) AS max(a)#x] ++- Expand [[a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x AS a#x, b#x AS b#x] + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias gstest1 + : +- View (`gstest1`, [a#x,b#x,v#x]) + : +- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS v#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.gstest2 + +- Relation spark_catalog.default.gstest2[a#x,b#x,c#x,d#x,e#x,f#x,g#x,h#x] parquet + + +-- !query +select t1.a, t2.b, grouping(t1.a), grouping(t2.b), sum(t1.v), max(t2.a) + from gstest1 t1 join gstest2 t2 on (t1.a=t2.a) + group by grouping sets ((t1.a, t2.b), ()) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, max(a#x) AS max(a)#x] ++- Expand [[a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x AS a#x, b#x AS b#x] + +- Join Inner, (a#x = a#x) + :- SubqueryAlias t1 + : +- SubqueryAlias gstest1 + : +- View (`gstest1`, [a#x,b#x,v#x]) + : +- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS v#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.gstest2 + +- Relation spark_catalog.default.gstest2[a#x,b#x,c#x,d#x,e#x,f#x,g#x,h#x] parquet + + +-- !query +select a, b, grouping(a), grouping(b), sum(t1.v), max(t2.c) + from gstest1 t1 join gstest2 t2 using (a,b) + group by grouping sets ((a, b), ()) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, max(c#x) AS max(c)#x] ++- Expand [[a#x, b#x, v#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, 0], [a#x, b#x, v#x, c#x, d#x, e#x, f#x, g#x, h#x, null, null, 3]], [a#x, b#x, v#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, v#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x AS a#x, b#x AS b#x] + +- Project [a#x, b#x, v#x, c#x, d#x, e#x, f#x, g#x, h#x] + +- Join Inner, ((a#x = a#x) AND (b#x = b#x)) + :- SubqueryAlias t1 + : +- SubqueryAlias gstest1 + : +- View (`gstest1`, [a#x,b#x,v#x]) + : +- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS v#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.gstest2 + +- Relation spark_catalog.default.gstest2[a#x,b#x,c#x,d#x,e#x,f#x,g#x,h#x] parquet + + +-- !query +select four, x + from (select four, ten, 'foo' as x from tenk1) as t + group by grouping sets (four, x) + having x = 'foo' +-- !query analysis +Filter (x#x = foo) ++- Aggregate [four#x, x#x, spark_grouping_id#xL], [four#x, x#x] + +- Expand [[four#x, ten#x, x#x, four#x, null, 1], [four#x, ten#x, x#x, null, x#x, 2]], [four#x, ten#x, x#x, four#x, x#x, spark_grouping_id#xL] + +- Project [four#x, ten#x, x#x, four#x AS four#x, x#x AS x#x] + +- SubqueryAlias t + +- Project [four#x, ten#x, foo AS x#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select four, x || 'x' + from (select four, ten, 'foo' as x from tenk1) as t + group by grouping sets (four, x) + order by four +-- !query analysis +Sort [four#x ASC NULLS FIRST], true ++- Aggregate [four#x, x#x, spark_grouping_id#xL], [four#x, concat(x#x, x) AS concat(x, x)#x] + +- Expand [[four#x, ten#x, x#x, four#x, null, 1], [four#x, ten#x, x#x, null, x#x, 2]], [four#x, ten#x, x#x, four#x, x#x, spark_grouping_id#xL] + +- Project [four#x, ten#x, x#x, four#x AS four#x, x#x AS x#x] + +- SubqueryAlias t + +- Project [four#x, ten#x, foo AS x#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select (x+y)*1, sum(z) + from (select 1 as x, 2 as y, 3 as z) s + group by grouping sets (x+y, x) +-- !query analysis +Aggregate [(x#x + y#x)#x, x#x, spark_grouping_id#xL], [((x#x + y#x)#x * 1) AS ((x + y) * 1)#x, sum(z#x) AS sum(z)#xL] ++- Expand [[x#x, y#x, z#x, (x#x + y#x)#x, null, 1], [x#x, y#x, z#x, null, x#x, 2]], [x#x, y#x, z#x, (x#x + y#x)#x, x#x, spark_grouping_id#xL] + +- Project [x#x, y#x, z#x, (x#x + y#x) AS (x#x + y#x)#x, x#x AS x#x] + +- SubqueryAlias s + +- Project [1 AS x#x, 2 AS y#x, 3 AS z#x] + +- OneRowRelation + + +-- !query +CREATE TEMP VIEW int8_tbl AS SELECT * FROM VALUES + (123L, 456L), + (123L, 4567890123456789L), + (4567890123456789L, 123L), + (4567890123456789L, 4567890123456789L), + (4567890123456789L, -4567890123456789L) as int8_tbl(q1, q2) +-- !query analysis +CreateViewCommand `int8_tbl`, SELECT * FROM VALUES + (123L, 456L), + (123L, 4567890123456789L), + (4567890123456789L, 123L), + (4567890123456789L, 4567890123456789L), + (4567890123456789L, -4567890123456789L) as int8_tbl(q1, q2), false, false, LocalTempView, true + +- Project [q1#xL, q2#xL] + +- SubqueryAlias int8_tbl + +- LocalRelation [q1#xL, q2#xL] + + +-- !query +select x, not x as not_x, q2 from + (select *, q1 = 1 as x from int8_tbl i1) as t + group by grouping sets(x, q2) + order by x, q2 +-- !query analysis +Sort [x#x ASC NULLS FIRST, q2#xL ASC NULLS FIRST], true ++- Aggregate [x#x, q2#xL, spark_grouping_id#xL], [x#x, NOT x#x AS not_x#x, q2#xL] + +- Expand [[q1#xL, q2#xL, x#x, x#x, null, 1], [q1#xL, q2#xL, x#x, null, q2#xL, 2]], [q1#xL, q2#xL, x#x, x#x, q2#xL, spark_grouping_id#xL] + +- Project [q1#xL, q2#xL, x#x, x#x AS x#x, q2#xL AS q2#xL] + +- SubqueryAlias t + +- Project [q1#xL, q2#xL, (q1#xL = cast(1 as bigint)) AS x#x] + +- SubqueryAlias i1 + +- SubqueryAlias int8_tbl + +- View (`int8_tbl`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias int8_tbl + +- LocalRelation [q1#xL, q2#xL] + + +-- !query +DROP VIEW int8_tbl +-- !query analysis +DropTempViewCommand int8_tbl + + +-- !query +select ten, sum(distinct four) from onek a +group by grouping sets((ten,four),(ten)) +having exists (select 1 from onek b where sum(distinct a.four) = b.four) +-- !query analysis +Filter exists#x [sum(DISTINCT four)#xL] +: +- Project [1 AS 1#x] +: +- Filter (outer(sum(DISTINCT four)#xL) = cast(four#x as bigint)) +: +- SubqueryAlias b +: +- SubqueryAlias spark_catalog.default.onek +: +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet ++- Aggregate [ten#x, four#x, spark_grouping_id#xL], [ten#x, sum(distinct four#x) AS sum(DISTINCT four)#xL] + +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, four#x, 0], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, null, 1]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, four#x, spark_grouping_id#xL] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x, four#x AS four#x] + +- SubqueryAlias a + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select a,count(*) from gstest2 group by rollup(a) order by a +-- !query analysis +Sort [a#x ASC NULLS FIRST], true ++- Aggregate [a#x, spark_grouping_id#xL], [a#x, count(1) AS count(1)#xL] + +- Expand [[a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, 0], [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, null, 1]], [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, spark_grouping_id#xL] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x AS a#x] + +- SubqueryAlias spark_catalog.default.gstest2 + +- Relation spark_catalog.default.gstest2[a#x,b#x,c#x,d#x,e#x,f#x,g#x,h#x] parquet + + +-- !query +select a,count(*) from gstest2 group by rollup(a) having a is distinct from 1 order by a +-- !query analysis +Sort [a#x ASC NULLS FIRST], true ++- Filter NOT (a#x <=> 1) + +- Aggregate [a#x, spark_grouping_id#xL], [a#x, count(1) AS count(1)#xL] + +- Expand [[a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, 0], [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, null, 1]], [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, spark_grouping_id#xL] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x AS a#x] + +- SubqueryAlias spark_catalog.default.gstest2 + +- Relation spark_catalog.default.gstest2[a#x,b#x,c#x,d#x,e#x,f#x,g#x,h#x] parquet + + +-- !query +select ten, grouping(ten) from onek +group by grouping sets(ten) having grouping(ten) >= 0 +order by 2,1 +-- !query analysis +Sort [grouping(ten)#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true ++- Project [ten#x, grouping(ten)#x] + +- Filter (cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) >= 0) + +- Aggregate [ten#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL] + +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, 0]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, spark_grouping_id#xL] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select ten, grouping(ten) from onek +group by grouping sets(ten, four) having grouping(ten) > 0 +order by 2,1 +-- !query analysis +Sort [grouping(ten)#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true ++- Project [ten#x, grouping(ten)#x] + +- Filter (cast(cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) as int) > 0) + +- Aggregate [ten#x, four#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL] + +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, null, 1], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, null, four#x, 2]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, four#x, spark_grouping_id#xL] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x, four#x AS four#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select ten, grouping(ten) from onek +group by rollup(ten) having grouping(ten) > 0 +order by 2,1 +-- !query analysis +Sort [grouping(ten)#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true ++- Project [ten#x, grouping(ten)#x] + +- Filter (cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) > 0) + +- Aggregate [ten#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL] + +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, 0], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, null, 1]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, spark_grouping_id#xL] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select ten, grouping(ten) from onek +group by cube(ten) having grouping(ten) > 0 +order by 2,1 +-- !query analysis +Sort [grouping(ten)#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true ++- Project [ten#x, grouping(ten)#x] + +- Filter (cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) > 0) + +- Aggregate [ten#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL] + +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, 0], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, null, 1]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, spark_grouping_id#xL] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select ten, sum(distinct four) filter (where string(four) like '123') from onek a +group by rollup(ten) +-- !query analysis +Aggregate [ten#x, spark_grouping_id#xL], [ten#x, sum(distinct four#x) FILTER (WHERE cast(four#x as string) LIKE 123) AS sum(DISTINCT four) FILTER (WHERE four LIKE 123)#xL] ++- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, 0], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, null, 1]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, spark_grouping_id#xL] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x] + +- SubqueryAlias a + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select count(*) from gstest4 group by rollup(unhashable_col,unsortable_col) +-- !query analysis +Aggregate [unhashable_col#x, unsortable_col#x, spark_grouping_id#xL], [count(1) AS count(1)#xL] ++- Expand [[id#x, v#x, unhashable_col#x, unsortable_col#x, unhashable_col#x, unsortable_col#x, 0], [id#x, v#x, unhashable_col#x, unsortable_col#x, unhashable_col#x, null, 1], [id#x, v#x, unhashable_col#x, unsortable_col#x, null, null, 3]], [id#x, v#x, unhashable_col#x, unsortable_col#x, unhashable_col#x, unsortable_col#x, spark_grouping_id#xL] + +- Project [id#x, v#x, unhashable_col#x, unsortable_col#x, unhashable_col#x AS unhashable_col#x, unsortable_col#x AS unsortable_col#x] + +- SubqueryAlias spark_catalog.default.gstest4 + +- Relation spark_catalog.default.gstest4[id#x,v#x,unhashable_col#x,unsortable_col#x] parquet + + +-- !query +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by grouping sets ((a),(b)) order by 3,4,1,2 /* 3,1,2 */ +-- !query analysis +Sort [grouping(a)#x ASC NULLS FIRST, grouping(b)#x ASC NULLS FIRST, a#x ASC NULLS FIRST, b#x ASC NULLS FIRST], true ++- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] + +- Expand [[a#x, b#x, v#x, a#x, null, 1], [a#x, b#x, v#x, null, b#x, 2]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias gstest1 + +- View (`gstest1`, [a#x,b#x,v#x]) + +- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS v#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by cube(a,b) order by 3,4,1,2 /* 3,1,2 */ +-- !query analysis +Sort [grouping(a)#x ASC NULLS FIRST, grouping(b)#x ASC NULLS FIRST, a#x ASC NULLS FIRST, b#x ASC NULLS FIRST], true ++- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] + +- Expand [[a#x, b#x, v#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, null, 1], [a#x, b#x, v#x, null, b#x, 2], [a#x, b#x, v#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias gstest1 + +- View (`gstest1`, [a#x,b#x,v#x]) + +- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS v#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +select unsortable_col, count(*) + from gstest4 group by grouping sets ((unsortable_col),(unsortable_col)) + order by string(unsortable_col) +-- !query analysis +Sort [cast(unsortable_col#x as string) ASC NULLS FIRST], true ++- Aggregate [unsortable_col#x, spark_grouping_id#xL, _gen_grouping_pos#x], [unsortable_col#x, count(1) AS count(1)#xL] + +- Expand [[id#x, v#x, unhashable_col#x, unsortable_col#x, unsortable_col#x, 0, 0], [id#x, v#x, unhashable_col#x, unsortable_col#x, unsortable_col#x, 0, 1]], [id#x, v#x, unhashable_col#x, unsortable_col#x, unsortable_col#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [id#x, v#x, unhashable_col#x, unsortable_col#x, unsortable_col#x AS unsortable_col#x] + +- SubqueryAlias spark_catalog.default.gstest4 + +- Relation spark_catalog.default.gstest4[id#x,v#x,unhashable_col#x,unsortable_col#x] parquet + + +-- !query +select unhashable_col, unsortable_col, + grouping(unhashable_col), grouping(unsortable_col), + count(*), sum(v) + from gstest4 group by grouping sets ((unhashable_col),(unsortable_col)) + order by 3, 4, 6 /* 3, 5 */ +-- !query analysis +Sort [grouping(unhashable_col)#x ASC NULLS FIRST, grouping(unsortable_col)#x ASC NULLS FIRST, sum(v)#xL ASC NULLS FIRST], true ++- Aggregate [unhashable_col#x, unsortable_col#x, spark_grouping_id#xL], [unhashable_col#x, unsortable_col#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(unhashable_col)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(unsortable_col)#x, count(1) AS count(1)#xL, sum(v#x) AS sum(v)#xL] + +- Expand [[id#x, v#x, unhashable_col#x, unsortable_col#x, unhashable_col#x, null, 1], [id#x, v#x, unhashable_col#x, unsortable_col#x, null, unsortable_col#x, 2]], [id#x, v#x, unhashable_col#x, unsortable_col#x, unhashable_col#x, unsortable_col#x, spark_grouping_id#xL] + +- Project [id#x, v#x, unhashable_col#x, unsortable_col#x, unhashable_col#x AS unhashable_col#x, unsortable_col#x AS unsortable_col#x] + +- SubqueryAlias spark_catalog.default.gstest4 + +- Relation spark_catalog.default.gstest4[id#x,v#x,unhashable_col#x,unsortable_col#x] parquet + + +-- !query +select unhashable_col, unsortable_col, + grouping(unhashable_col), grouping(unsortable_col), + count(*), sum(v) + from gstest4 group by grouping sets ((v,unhashable_col),(v,unsortable_col)) + order by 3, 4, 6 /* 3,5 */ +-- !query analysis +Sort [grouping(unhashable_col)#x ASC NULLS FIRST, grouping(unsortable_col)#x ASC NULLS FIRST, sum(v)#xL ASC NULLS FIRST], true ++- Aggregate [v#x, unhashable_col#x, unsortable_col#x, spark_grouping_id#xL], [unhashable_col#x, unsortable_col#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(unhashable_col)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(unsortable_col)#x, count(1) AS count(1)#xL, sum(v#x) AS sum(v)#xL] + +- Expand [[id#x, v#x, unhashable_col#x, unsortable_col#x, v#x, unhashable_col#x, null, 1], [id#x, v#x, unhashable_col#x, unsortable_col#x, v#x, null, unsortable_col#x, 2]], [id#x, v#x, unhashable_col#x, unsortable_col#x, v#x, unhashable_col#x, unsortable_col#x, spark_grouping_id#xL] + +- Project [id#x, v#x, unhashable_col#x, unsortable_col#x, v#x AS v#x, unhashable_col#x AS unhashable_col#x, unsortable_col#x AS unsortable_col#x] + +- SubqueryAlias spark_catalog.default.gstest4 + +- Relation spark_catalog.default.gstest4[id#x,v#x,unhashable_col#x,unsortable_col#x] parquet + + +-- !query +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),a) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, v#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, null, 1]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias spark_catalog.default.gstest_empty + +- Relation spark_catalog.default.gstest_empty[a#x,b#x,v#x] parquet + + +-- !query +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),()) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, v#x, a#x, b#x, 0], [a#x, b#x, v#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias spark_catalog.default.gstest_empty + +- Relation spark_catalog.default.gstest_empty[a#x,b#x,v#x] parquet + + +-- !query +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),(),(),()) +-- !query analysis +Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, v#x, a#x, b#x, 0, 0], [a#x, b#x, v#x, null, null, 3, 1], [a#x, b#x, v#x, null, null, 3, 2], [a#x, b#x, v#x, null, null, 3, 3]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias spark_catalog.default.gstest_empty + +- Relation spark_catalog.default.gstest_empty[a#x,b#x,v#x] parquet + + +-- !query +select sum(v), count(*) from gstest_empty group by grouping sets ((),(),()) +-- !query analysis +Aggregate [spark_grouping_id#xL, _gen_grouping_pos#x], [sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL] ++- Expand [[a#x, b#x, v#x, 0, 0], [a#x, b#x, v#x, 0, 1], [a#x, b#x, v#x, 0, 2]], [a#x, b#x, v#x, spark_grouping_id#xL, _gen_grouping_pos#x] + +- Project [a#x, b#x, v#x] + +- SubqueryAlias spark_catalog.default.gstest_empty + +- Relation spark_catalog.default.gstest_empty[a#x,b#x,v#x] parquet + + +-- !query +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by grouping sets ((a,b),(a+1,b+1),(a+2,b+2)) order by 3,4,7 /* 3,6 */ +-- !query analysis +Sort [grouping(a)#x ASC NULLS FIRST, grouping(b)#x ASC NULLS FIRST, max(v)#x ASC NULLS FIRST], true ++- Aggregate [a#x, b#x, (a#x + 1)#x, (b#x + 1)#x, (a#x + 2)#x, (b#x + 2)#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 5) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 4) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] + +- Expand [[a#x, b#x, v#x, a#x, b#x, null, null, null, null, 15], [a#x, b#x, v#x, null, null, (a#x + 1)#x, (b#x + 1)#x, null, null, 51], [a#x, b#x, v#x, null, null, null, null, (a#x + 2)#x, (b#x + 2)#x, 60]], [a#x, b#x, v#x, a#x, b#x, (a#x + 1)#x, (b#x + 1)#x, (a#x + 2)#x, (b#x + 2)#x, spark_grouping_id#xL] + +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x, (a#x + 1) AS (a#x + 1)#x, (b#x + 1) AS (b#x + 1)#x, (a#x + 2) AS (a#x + 2)#x, (b#x + 2) AS (b#x + 2)#x] + +- SubqueryAlias gstest1 + +- View (`gstest1`, [a#x,b#x,v#x]) + +- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS v#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +select a, b, sum(c), sum(sum(c)) over (order by a,b) as rsum + from gstest2 group by cube (a,b) order by rsum, a, b +-- !query analysis +Sort [rsum#xL ASC NULLS FIRST, a#x ASC NULLS FIRST, b#x ASC NULLS FIRST], true ++- Project [a#x, b#x, sum(c)#xL, rsum#xL] + +- Project [a#x, b#x, sum(c)#xL, _w0#xL, rsum#xL, rsum#xL] + +- Window [sum(_w0#xL) windowspecdefinition(a#x ASC NULLS FIRST, b#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS rsum#xL], [a#x ASC NULLS FIRST, b#x ASC NULLS FIRST] + +- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, sum(c#x) AS sum(c)#xL, sum(c#x) AS _w0#xL] + +- Expand [[a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, 0], [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, null, 1], [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, null, b#x, 2], [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, null, null, 3]], [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias spark_catalog.default.gstest2 + +- Relation spark_catalog.default.gstest2[a#x,b#x,c#x,d#x,e#x,f#x,g#x,h#x] parquet + + +-- !query +SELECT a, b, count(*), max(a), max(b) FROM gstest3 GROUP BY GROUPING SETS(a, b,()) ORDER BY a, b +-- !query analysis +Sort [a#x ASC NULLS FIRST, b#x ASC NULLS FIRST], true ++- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, count(1) AS count(1)#xL, max(a#x) AS max(a)#x, max(b#x) AS max(b)#x] + +- Expand [[a#x, b#x, c#x, d#x, a#x, null, 1], [a#x, b#x, c#x, d#x, null, b#x, 2], [a#x, b#x, c#x, d#x, null, null, 3]], [a#x, b#x, c#x, d#x, a#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, c#x, d#x, a#x AS a#x, b#x AS b#x] + +- SubqueryAlias spark_catalog.default.gstest3 + +- Relation spark_catalog.default.gstest3[a#x,b#x,c#x,d#x] parquet + + +-- !query +select v||'a', case grouping(v||'a') when 1 then 1 else 0 end, count(*) + from values (1, 'a'), (1, 'b') u(i,v) + group by rollup(i, v||'a') order by 1,3 +-- !query analysis +Sort [concat(v, a)#x ASC NULLS FIRST, count(1)#xL ASC NULLS FIRST], true ++- Aggregate [i#x, concat(v#x, a)#x, spark_grouping_id#xL], [concat(v#x, a)#x AS concat(v, a)#x, CASE WHEN (cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) = 1) THEN 1 ELSE 0 END AS CASE WHEN (grouping(concat(v, a)) = 1) THEN 1 ELSE 0 END#x, count(1) AS count(1)#xL] + +- Expand [[i#x, v#x, i#x, concat(v#x, a)#x, 0], [i#x, v#x, i#x, null, 1], [i#x, v#x, null, null, 3]], [i#x, v#x, i#x, concat(v#x, a)#x, spark_grouping_id#xL] + +- Project [i#x, v#x, i#x AS i#x, concat(v#x, a) AS concat(v#x, a)#x] + +- SubqueryAlias u + +- LocalRelation [i#x, v#x] + + +-- !query +select v||'a', case when grouping(v||'a') = 1 then 1 else 0 end, count(*) + from values (1, 'a'), (1, 'b') u(i,v) + group by rollup(i, v||'a') order by 1,3 +-- !query analysis +Sort [concat(v, a)#x ASC NULLS FIRST, count(1)#xL ASC NULLS FIRST], true ++- Aggregate [i#x, concat(v#x, a)#x, spark_grouping_id#xL], [concat(v#x, a)#x AS concat(v, a)#x, CASE WHEN (cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) = 1) THEN 1 ELSE 0 END AS CASE WHEN (grouping(concat(v, a)) = 1) THEN 1 ELSE 0 END#x, count(1) AS count(1)#xL] + +- Expand [[i#x, v#x, i#x, concat(v#x, a)#x, 0], [i#x, v#x, i#x, null, 1], [i#x, v#x, null, null, 3]], [i#x, v#x, i#x, concat(v#x, a)#x, spark_grouping_id#xL] + +- Project [i#x, v#x, i#x AS i#x, concat(v#x, a) AS concat(v#x, a)#x] + +- SubqueryAlias u + +- LocalRelation [i#x, v#x] + + +-- !query +DROP VIEW gstest1 +-- !query analysis +DropTempViewCommand gstest1 + + +-- !query +DROP TABLE gstest2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`gstest2`, false, false, false + + +-- !query +DROP TABLE gstest3 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`gstest3`, false, false, false + + +-- !query +DROP TABLE gstest4 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`gstest4`, false, false, false + + +-- !query +DROP TABLE gstest_empty +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`gstest_empty`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/insert.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/insert.sql.out new file mode 100644 index 0000000000000..3d96f8aa64168 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/insert.sql.out @@ -0,0 +1,66 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create table inserttest (col1 int, col2 int /* NOT NULL */, col3 string /* default 'testing' */) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`inserttest`, false + + +-- !query +insert into inserttest values (NULL, 3, 'testing') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/inserttest, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/inserttest], Append, `spark_catalog`.`default`.`inserttest`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/inserttest), [col1, col2, col3] ++- Project [cast(col1#x as int) AS col1#x, col2#x, col3#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into inserttest values (NULL, 5, 'testing') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/inserttest, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/inserttest], Append, `spark_catalog`.`default`.`inserttest`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/inserttest), [col1, col2, col3] ++- Project [cast(col1#x as int) AS col1#x, col2#x, col3#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into inserttest values (NULL, 5, 'test') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/inserttest, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/inserttest], Append, `spark_catalog`.`default`.`inserttest`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/inserttest), [col1, col2, col3] ++- Project [cast(col1#x as int) AS col1#x, col2#x, col3#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into inserttest values (NULL, 7, 'testing') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/inserttest, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/inserttest], Append, `spark_catalog`.`default`.`inserttest`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/inserttest), [col1, col2, col3] ++- Project [cast(col1#x as int) AS col1#x, col2#x, col3#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +select * from inserttest +-- !query analysis +Project [col1#x, col2#x, col3#x] ++- SubqueryAlias spark_catalog.default.inserttest + +- Relation spark_catalog.default.inserttest[col1#x,col2#x,col3#x] parquet + + +-- !query +insert into inserttest values(30, 50, repeat('x', 10000)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/inserttest, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/inserttest], Append, `spark_catalog`.`default`.`inserttest`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/inserttest), [col1, col2, col3] ++- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +select col1, col2, char_length(col3) from inserttest +-- !query analysis +Project [col1#x, col2#x, char_length(col3#x) AS char_length(col3)#x] ++- SubqueryAlias spark_catalog.default.inserttest + +- Relation spark_catalog.default.inserttest[col1#x,col2#x,col3#x] parquet + + +-- !query +drop table inserttest +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`inserttest`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int2.sql.out new file mode 100644 index 0000000000000..58c8904e91d4d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int2.sql.out @@ -0,0 +1,330 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE INT2_TBL(f1 smallint) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`INT2_TBL`, false + + +-- !query +INSERT INTO INT2_TBL VALUES (smallint(trim('0 '))) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int2_tbl], Append, `spark_catalog`.`default`.`int2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int2_tbl), [f1] ++- Project [cast(col1#x as smallint) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO INT2_TBL VALUES (smallint(trim(' 1234 '))) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int2_tbl], Append, `spark_catalog`.`default`.`int2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int2_tbl), [f1] ++- Project [cast(col1#x as smallint) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO INT2_TBL VALUES (smallint(trim(' -1234'))) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int2_tbl], Append, `spark_catalog`.`default`.`int2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int2_tbl), [f1] ++- Project [cast(col1#x as smallint) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO INT2_TBL VALUES (smallint('32767')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int2_tbl], Append, `spark_catalog`.`default`.`int2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int2_tbl), [f1] ++- Project [cast(col1#x as smallint) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO INT2_TBL VALUES (smallint('-32767')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int2_tbl], Append, `spark_catalog`.`default`.`int2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int2_tbl), [f1] ++- Project [cast(col1#x as smallint) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT '' AS five, * FROM INT2_TBL +-- !query analysis +Project [ AS five#x, f1#x] ++- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> smallint('0') +-- !query analysis +Project [ AS four#x, f1#x] ++- Filter NOT (f1#x = cast(0 as smallint)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> int('0') +-- !query analysis +Project [ AS four#x, f1#x] ++- Filter NOT (cast(f1#x as int) = cast(0 as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = smallint('0') +-- !query analysis +Project [ AS one#x, f1#x] ++- Filter (f1#x = cast(0 as smallint)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = int('0') +-- !query analysis +Project [ AS one#x, f1#x] ++- Filter (cast(f1#x as int) = cast(0 as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < smallint('0') +-- !query analysis +Project [ AS two#x, f1#x] ++- Filter (f1#x < cast(0 as smallint)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < int('0') +-- !query analysis +Project [ AS two#x, f1#x] ++- Filter (cast(f1#x as int) < cast(0 as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= smallint('0') +-- !query analysis +Project [ AS three#x, f1#x] ++- Filter (f1#x <= cast(0 as smallint)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= int('0') +-- !query analysis +Project [ AS three#x, f1#x] ++- Filter (cast(f1#x as int) <= cast(0 as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > smallint('0') +-- !query analysis +Project [ AS two#x, f1#x] ++- Filter (f1#x > cast(0 as smallint)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > int('0') +-- !query analysis +Project [ AS two#x, f1#x] ++- Filter (cast(f1#x as int) > cast(0 as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= smallint('0') +-- !query analysis +Project [ AS three#x, f1#x] ++- Filter (f1#x >= cast(0 as smallint)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= int('0') +-- !query analysis +Project [ AS three#x, f1#x] ++- Filter (cast(f1#x as int) >= cast(0 as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS one, i.* FROM INT2_TBL i WHERE (i.f1 % smallint('2')) = smallint('1') +-- !query analysis +Project [ AS one#x, f1#x] ++- Filter ((f1#x % cast(2 as smallint)) = cast(1 as smallint)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, i.* FROM INT2_TBL i WHERE (i.f1 % int('2')) = smallint('0') +-- !query analysis +Project [ AS three#x, f1#x] ++- Filter ((cast(f1#x as int) % cast(2 as int)) = cast(cast(0 as smallint) as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT2_TBL i +WHERE abs(f1) < 16384 +-- !query analysis +Project [ AS five#x, f1#x, (f1#x * cast(2 as smallint)) AS x#x] ++- Filter (cast(abs(f1#x) as int) < 16384) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT2_TBL i +-- !query analysis +Project [ AS five#x, f1#x, (cast(f1#x as int) * cast(2 as int)) AS x#x] ++- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT2_TBL i +WHERE f1 < 32766 +-- !query analysis +Project [ AS five#x, f1#x, (f1#x + cast(2 as smallint)) AS x#x] ++- Filter (cast(f1#x as int) < 32766) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT2_TBL i +-- !query analysis +Project [ AS five#x, f1#x, (cast(f1#x as int) + cast(2 as int)) AS x#x] ++- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT2_TBL i +WHERE f1 > -32767 +-- !query analysis +Project [ AS five#x, f1#x, (f1#x - cast(2 as smallint)) AS x#x] ++- Filter (cast(f1#x as int) > -32767) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT2_TBL i +-- !query analysis +Project [ AS five#x, f1#x, (cast(f1#x as int) - cast(2 as int)) AS x#x] ++- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT2_TBL i +-- !query analysis +Project [ AS five#x, f1#x, (cast(f1#x as double) / cast(cast(2 as smallint) as double)) AS x#x] ++- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT2_TBL i +-- !query analysis +Project [ AS five#x, f1#x, (cast(f1#x as double) / cast(cast(2 as int) as double)) AS x#x] ++- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int2_tbl + +- Relation spark_catalog.default.int2_tbl[f1#x] parquet + + +-- !query +SELECT string(shiftleft(smallint(-1), 15)) +-- !query analysis +Project [cast(shiftleft(cast(cast(-1 as smallint) as int), 15) as string) AS shiftleft(-1, 15)#x] ++- OneRowRelation + + +-- !query +SELECT string(smallint(shiftleft(smallint(-1), 15))+1) +-- !query analysis +Project [cast((cast(cast(shiftleft(cast(cast(-1 as smallint) as int), 15) as smallint) as int) + 1) as string) AS (shiftleft(-1, 15) + 1)#x] ++- OneRowRelation + + +-- !query +SELECT smallint(-32768) % smallint(-1) +-- !query analysis +Project [(cast(-32768 as smallint) % cast(-1 as smallint)) AS (-32768 % -1)#x] ++- OneRowRelation + + +-- !query +SELECT x, smallint(x) AS int2_value +FROM (VALUES float(-2.5), + float(-1.5), + float(-0.5), + float(0.0), + float(0.5), + float(1.5), + float(2.5)) t(x) +-- !query analysis +Project [x#x, cast(x#x as smallint) AS int2_value#x] ++- SubqueryAlias t + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT x, smallint(x) AS int2_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x) +-- !query analysis +Project [x#x, cast(x#x as smallint) AS int2_value#x] ++- SubqueryAlias t + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +DROP TABLE INT2_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`INT2_TBL`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int4.sql.out new file mode 100644 index 0000000000000..9885ef0ef9140 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int4.sql.out @@ -0,0 +1,474 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE INT4_TBL(f1 int) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`INT4_TBL`, false + + +-- !query +INSERT INTO INT4_TBL VALUES (int(trim(' 0 '))) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int4_tbl], Append, `spark_catalog`.`default`.`int4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int4_tbl), [f1] ++- Project [cast(col1#x as int) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO INT4_TBL VALUES (int(trim('123456 '))) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int4_tbl], Append, `spark_catalog`.`default`.`int4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int4_tbl), [f1] ++- Project [cast(col1#x as int) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO INT4_TBL VALUES (int(trim(' -123456'))) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int4_tbl], Append, `spark_catalog`.`default`.`int4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int4_tbl), [f1] ++- Project [cast(col1#x as int) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO INT4_TBL VALUES (int('2147483647')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int4_tbl], Append, `spark_catalog`.`default`.`int4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int4_tbl), [f1] ++- Project [cast(col1#x as int) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO INT4_TBL VALUES (int('-2147483647')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int4_tbl], Append, `spark_catalog`.`default`.`int4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int4_tbl), [f1] ++- Project [cast(col1#x as int) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT '' AS five, * FROM INT4_TBL +-- !query analysis +Project [ AS five#x, f1#x] ++- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> smallint('0') +-- !query analysis +Project [ AS four#x, f1#x] ++- Filter NOT (f1#x = cast(cast(0 as smallint) as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> int('0') +-- !query analysis +Project [ AS four#x, f1#x] ++- Filter NOT (f1#x = cast(0 as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = smallint('0') +-- !query analysis +Project [ AS one#x, f1#x] ++- Filter (f1#x = cast(cast(0 as smallint) as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = int('0') +-- !query analysis +Project [ AS one#x, f1#x] ++- Filter (f1#x = cast(0 as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < smallint('0') +-- !query analysis +Project [ AS two#x, f1#x] ++- Filter (f1#x < cast(cast(0 as smallint) as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < int('0') +-- !query analysis +Project [ AS two#x, f1#x] ++- Filter (f1#x < cast(0 as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= smallint('0') +-- !query analysis +Project [ AS three#x, f1#x] ++- Filter (f1#x <= cast(cast(0 as smallint) as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= int('0') +-- !query analysis +Project [ AS three#x, f1#x] ++- Filter (f1#x <= cast(0 as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > smallint('0') +-- !query analysis +Project [ AS two#x, f1#x] ++- Filter (f1#x > cast(cast(0 as smallint) as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > int('0') +-- !query analysis +Project [ AS two#x, f1#x] ++- Filter (f1#x > cast(0 as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= smallint('0') +-- !query analysis +Project [ AS three#x, f1#x] ++- Filter (f1#x >= cast(cast(0 as smallint) as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= int('0') +-- !query analysis +Project [ AS three#x, f1#x] ++- Filter (f1#x >= cast(0 as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS one, i.* FROM INT4_TBL i WHERE (i.f1 % smallint('2')) = smallint('1') +-- !query analysis +Project [ AS one#x, f1#x] ++- Filter ((f1#x % cast(cast(2 as smallint) as int)) = cast(cast(1 as smallint) as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS three, i.* FROM INT4_TBL i WHERE (i.f1 % int('2')) = smallint('0') +-- !query analysis +Project [ AS three#x, f1#x] ++- Filter ((f1#x % cast(2 as int)) = cast(cast(0 as smallint) as int)) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i +-- !query analysis +Project [ AS five#x, f1#x, (f1#x * cast(cast(2 as smallint) as int)) AS x#x] ++- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i +WHERE abs(f1) < 1073741824 +-- !query analysis +Project [ AS five#x, f1#x, (f1#x * cast(cast(2 as smallint) as int)) AS x#x] ++- Filter (abs(f1#x) < 1073741824) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i +-- !query analysis +Project [ AS five#x, f1#x, (f1#x * cast(2 as int)) AS x#x] ++- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i +WHERE abs(f1) < 1073741824 +-- !query analysis +Project [ AS five#x, f1#x, (f1#x * cast(2 as int)) AS x#x] ++- Filter (abs(f1#x) < 1073741824) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i +-- !query analysis +Project [ AS five#x, f1#x, (f1#x + cast(cast(2 as smallint) as int)) AS x#x] ++- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i +WHERE f1 < 2147483646 +-- !query analysis +Project [ AS five#x, f1#x, (f1#x + cast(cast(2 as smallint) as int)) AS x#x] ++- Filter (f1#x < 2147483646) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i +-- !query analysis +Project [ AS five#x, f1#x, (f1#x + cast(2 as int)) AS x#x] ++- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i +WHERE f1 < 2147483646 +-- !query analysis +Project [ AS five#x, f1#x, (f1#x + cast(2 as int)) AS x#x] ++- Filter (f1#x < 2147483646) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i +-- !query analysis +Project [ AS five#x, f1#x, (f1#x - cast(cast(2 as smallint) as int)) AS x#x] ++- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i +WHERE f1 > -2147483647 +-- !query analysis +Project [ AS five#x, f1#x, (f1#x - cast(cast(2 as smallint) as int)) AS x#x] ++- Filter (f1#x > -2147483647) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i +-- !query analysis +Project [ AS five#x, f1#x, (f1#x - cast(2 as int)) AS x#x] ++- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i +WHERE f1 > -2147483647 +-- !query analysis +Project [ AS five#x, f1#x, (f1#x - cast(2 as int)) AS x#x] ++- Filter (f1#x > -2147483647) + +- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT4_TBL i +-- !query analysis +Project [ AS five#x, f1#x, (cast(f1#x as double) / cast(cast(2 as smallint) as double)) AS x#x] ++- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT4_TBL i +-- !query analysis +Project [ AS five#x, f1#x, (cast(f1#x as double) / cast(cast(2 as int) as double)) AS x#x] ++- SubqueryAlias i + +- SubqueryAlias spark_catalog.default.int4_tbl + +- Relation spark_catalog.default.int4_tbl[f1#x] parquet + + +-- !query +SELECT -2+3 AS one +-- !query analysis +Project [(-2 + 3) AS one#x] ++- OneRowRelation + + +-- !query +SELECT 4-2 AS two +-- !query analysis +Project [(4 - 2) AS two#x] ++- OneRowRelation + + +-- !query +SELECT 2- -1 AS three +-- !query analysis +Project [(2 - -1) AS three#x] ++- OneRowRelation + + +-- !query +SELECT 2 - -2 AS four +-- !query analysis +Project [(2 - -2) AS four#x] ++- OneRowRelation + + +-- !query +SELECT smallint('2') * smallint('2') = smallint('16') / smallint('4') AS true +-- !query analysis +Project [(cast((cast(2 as smallint) * cast(2 as smallint)) as double) = (cast(cast(16 as smallint) as double) / cast(cast(4 as smallint) as double))) AS true#x] ++- OneRowRelation + + +-- !query +SELECT int('2') * smallint('2') = smallint('16') / int('4') AS true +-- !query analysis +Project [(cast((cast(2 as int) * cast(cast(2 as smallint) as int)) as double) = (cast(cast(16 as smallint) as double) / cast(cast(4 as int) as double))) AS true#x] ++- OneRowRelation + + +-- !query +SELECT smallint('2') * int('2') = int('16') / smallint('4') AS true +-- !query analysis +Project [(cast((cast(cast(2 as smallint) as int) * cast(2 as int)) as double) = (cast(cast(16 as int) as double) / cast(cast(4 as smallint) as double))) AS true#x] ++- OneRowRelation + + +-- !query +SELECT int('1000') < int('999') AS `false` +-- !query analysis +Project [(cast(1000 as int) < cast(999 as int)) AS false#x] ++- OneRowRelation + + +-- !query +SELECT 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 AS ten +-- !query analysis +Project [(((((((((1 + 1) + 1) + 1) + 1) + 1) + 1) + 1) + 1) + 1) AS ten#x] ++- OneRowRelation + + +-- !query +SELECT 2 + 2 / 2 AS three +-- !query analysis +Project [(cast(2 as double) + (cast(2 as double) / cast(2 as double))) AS three#x] ++- OneRowRelation + + +-- !query +SELECT (2 + 2) / 2 AS two +-- !query analysis +Project [(cast((2 + 2) as double) / cast(2 as double)) AS two#x] ++- OneRowRelation + + +-- !query +SELECT string(shiftleft(int(-1), 31)) +-- !query analysis +Project [cast(shiftleft(cast(-1 as int), 31) as string) AS shiftleft(-1, 31)#x] ++- OneRowRelation + + +-- !query +SELECT string(int(shiftleft(int(-1), 31))+1) +-- !query analysis +Project [cast((cast(shiftleft(cast(-1 as int), 31) as int) + 1) as string) AS (shiftleft(-1, 31) + 1)#x] ++- OneRowRelation + + +-- !query +SELECT int(-2147483648) % int(-1) +-- !query analysis +Project [(cast(-2147483648 as int) % cast(-1 as int)) AS (-2147483648 % -1)#x] ++- OneRowRelation + + +-- !query +SELECT int(-2147483648) % smallint(-1) +-- !query analysis +Project [(cast(-2147483648 as int) % cast(cast(-1 as smallint) as int)) AS (-2147483648 % -1)#x] ++- OneRowRelation + + +-- !query +SELECT x, int(x) AS int4_value +FROM (VALUES double(-2.5), + double(-1.5), + double(-0.5), + double(0.0), + double(0.5), + double(1.5), + double(2.5)) t(x) +-- !query analysis +Project [x#x, cast(x#x as int) AS int4_value#x] ++- SubqueryAlias t + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT x, int(x) AS int4_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x) +-- !query analysis +Project [x#x, cast(x#x as int) AS int4_value#x] ++- SubqueryAlias t + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +DROP TABLE INT4_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`INT4_TBL`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int8.sql.out new file mode 100644 index 0000000000000..b1faa6ee9537b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int8.sql.out @@ -0,0 +1,771 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE INT8_TBL(q1 bigint, q2 bigint) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`INT8_TBL`, false + + +-- !query +INSERT INTO INT8_TBL VALUES(bigint(trim(' 123 ')),bigint(trim(' 456'))) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int8_tbl], Append, `spark_catalog`.`default`.`int8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int8_tbl), [q1, q2] ++- Project [cast(col1#xL as bigint) AS q1#xL, cast(col2#xL as bigint) AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +INSERT INTO INT8_TBL VALUES(bigint(trim('123 ')),bigint('4567890123456789')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int8_tbl], Append, `spark_catalog`.`default`.`int8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int8_tbl), [q1, q2] ++- Project [cast(col1#xL as bigint) AS q1#xL, cast(col2#xL as bigint) AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +INSERT INTO INT8_TBL VALUES(bigint('4567890123456789'),bigint('123')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int8_tbl], Append, `spark_catalog`.`default`.`int8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int8_tbl), [q1, q2] ++- Project [cast(col1#xL as bigint) AS q1#xL, cast(col2#xL as bigint) AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +INSERT INTO INT8_TBL VALUES(+4567890123456789,bigint('4567890123456789')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int8_tbl], Append, `spark_catalog`.`default`.`int8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int8_tbl), [q1, q2] ++- Project [cast(col1#xL as bigint) AS q1#xL, cast(col2#xL as bigint) AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +INSERT INTO INT8_TBL VALUES(bigint('+4567890123456789'),bigint('-4567890123456789')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int8_tbl], Append, `spark_catalog`.`default`.`int8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int8_tbl), [q1, q2] ++- Project [cast(col1#xL as bigint) AS q1#xL, cast(col2#xL as bigint) AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +SELECT * FROM INT8_TBL +-- !query analysis +Project [q1#xL, q2#xL] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 = 4567890123456789 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (q2#xL = 4567890123456789) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <> 4567890123456789 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter NOT (q2#xL = 4567890123456789) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 < 4567890123456789 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (q2#xL < 4567890123456789) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 > 4567890123456789 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (q2#xL > 4567890123456789) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <= 4567890123456789 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (q2#xL <= 4567890123456789) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 >= 4567890123456789 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (q2#xL >= 4567890123456789) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 = 456 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (q2#xL = cast(456 as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <> 456 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter NOT (q2#xL = cast(456 as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 < 456 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (q2#xL < cast(456 as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 > 456 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (q2#xL > cast(456 as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <= 456 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (q2#xL <= cast(456 as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 >= 456 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (q2#xL >= cast(456 as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 = q1 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (cast(123 as bigint) = q1#xL) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 <> q1 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter NOT (cast(123 as bigint) = q1#xL) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 < q1 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (cast(123 as bigint) < q1#xL) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 > q1 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (cast(123 as bigint) > q1#xL) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 <= q1 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (cast(123 as bigint) <= q1#xL) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 >= q1 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (cast(123 as bigint) >= q1#xL) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 = smallint('456') +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (q2#xL = cast(cast(456 as smallint) as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <> smallint('456') +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter NOT (q2#xL = cast(cast(456 as smallint) as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 < smallint('456') +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (q2#xL < cast(cast(456 as smallint) as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 > smallint('456') +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (q2#xL > cast(cast(456 as smallint) as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <= smallint('456') +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (q2#xL <= cast(cast(456 as smallint) as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 >= smallint('456') +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (q2#xL >= cast(cast(456 as smallint) as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') = q1 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (cast(cast(123 as smallint) as bigint) = q1#xL) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') <> q1 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter NOT (cast(cast(123 as smallint) as bigint) = q1#xL) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') < q1 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (cast(cast(123 as smallint) as bigint) < q1#xL) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') > q1 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (cast(cast(123 as smallint) as bigint) > q1#xL) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') <= q1 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (cast(cast(123 as smallint) as bigint) <= q1#xL) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') >= q1 +-- !query analysis +Project [q1#xL, q2#xL] ++- Filter (cast(cast(123 as smallint) as bigint) >= q1#xL) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS five, q1 AS plus, -q1 AS `minus` FROM INT8_TBL +-- !query analysis +Project [ AS five#x, q1#xL AS plus#xL, -q1#xL AS minus#xL] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS five, q1, q2, q1 + q2 AS plus FROM INT8_TBL +-- !query analysis +Project [ AS five#x, q1#xL, q2#xL, (q1#xL + q2#xL) AS plus#xL] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS five, q1, q2, q1 - q2 AS `minus` FROM INT8_TBL +-- !query analysis +Project [ AS five#x, q1#xL, q2#xL, (q1#xL - q2#xL) AS minus#xL] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL +-- !query analysis +Project [ AS three#x, q1#xL, q2#xL, (q1#xL * q2#xL) AS multiply#xL] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL + WHERE q1 < 1000 or (q2 > 0 and q2 < 1000) +-- !query analysis +Project [ AS three#x, q1#xL, q2#xL, (q1#xL * q2#xL) AS multiply#xL] ++- Filter ((q1#xL < cast(1000 as bigint)) OR ((q2#xL > cast(0 as bigint)) AND (q2#xL < cast(1000 as bigint)))) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS five, q1, q2, q1 / q2 AS divide, q1 % q2 AS mod FROM INT8_TBL +-- !query analysis +Project [ AS five#x, q1#xL, q2#xL, (cast(q1#xL as double) / cast(q2#xL as double)) AS divide#x, (q1#xL % q2#xL) AS mod#xL] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS five, q1, double(q1) FROM INT8_TBL +-- !query analysis +Project [ AS five#x, q1#xL, cast(q1#xL as double) AS q1#x] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS five, q2, double(q2) FROM INT8_TBL +-- !query analysis +Project [ AS five#x, q2#xL, cast(q2#xL as double) AS q2#x] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT 37 + q1 AS plus4 FROM INT8_TBL +-- !query analysis +Project [(cast(37 as bigint) + q1#xL) AS plus4#xL] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT 37 - q1 AS minus4 FROM INT8_TBL +-- !query analysis +Project [(cast(37 as bigint) - q1#xL) AS minus4#xL] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS five, 2 * q1 AS `twice int4` FROM INT8_TBL +-- !query analysis +Project [ AS five#x, (cast(2 as bigint) * q1#xL) AS twice int4#xL] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS five, q1 * 2 AS `twice int4` FROM INT8_TBL +-- !query analysis +Project [ AS five#x, (q1#xL * cast(2 as bigint)) AS twice int4#xL] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT q1 + int(42) AS `8plus4`, q1 - int(42) AS `8minus4`, q1 * int(42) AS `8mul4`, q1 / int(42) AS `8div4` FROM INT8_TBL +-- !query analysis +Project [(q1#xL + cast(cast(42 as int) as bigint)) AS 8plus4#xL, (q1#xL - cast(cast(42 as int) as bigint)) AS 8minus4#xL, (q1#xL * cast(cast(42 as int) as bigint)) AS 8mul4#xL, (cast(q1#xL as double) / cast(cast(42 as int) as double)) AS 8div4#x] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT int(246) + q1 AS `4plus8`, int(246) - q1 AS `4minus8`, int(246) * q1 AS `4mul8`, int(246) / q1 AS `4div8` FROM INT8_TBL +-- !query analysis +Project [(cast(cast(246 as int) as bigint) + q1#xL) AS 4plus8#xL, (cast(cast(246 as int) as bigint) - q1#xL) AS 4minus8#xL, (cast(cast(246 as int) as bigint) * q1#xL) AS 4mul8#xL, (cast(cast(246 as int) as double) / cast(q1#xL as double)) AS 4div8#x] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT q1 + smallint(42) AS `8plus2`, q1 - smallint(42) AS `8minus2`, q1 * smallint(42) AS `8mul2`, q1 / smallint(42) AS `8div2` FROM INT8_TBL +-- !query analysis +Project [(q1#xL + cast(cast(42 as smallint) as bigint)) AS 8plus2#xL, (q1#xL - cast(cast(42 as smallint) as bigint)) AS 8minus2#xL, (q1#xL * cast(cast(42 as smallint) as bigint)) AS 8mul2#xL, (cast(q1#xL as double) / cast(cast(42 as smallint) as double)) AS 8div2#x] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT smallint(246) + q1 AS `2plus8`, smallint(246) - q1 AS `2minus8`, smallint(246) * q1 AS `2mul8`, smallint(246) / q1 AS `2div8` FROM INT8_TBL +-- !query analysis +Project [(cast(cast(246 as smallint) as bigint) + q1#xL) AS 2plus8#xL, (cast(cast(246 as smallint) as bigint) - q1#xL) AS 2minus8#xL, (cast(cast(246 as smallint) as bigint) * q1#xL) AS 2mul8#xL, (cast(cast(246 as smallint) as double) / cast(q1#xL as double)) AS 2div8#x] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT q2, abs(q2) FROM INT8_TBL +-- !query analysis +Project [q2#xL, abs(q2#xL) AS abs(q2)#xL] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT min(q1), min(q2) FROM INT8_TBL +-- !query analysis +Aggregate [min(q1#xL) AS min(q1)#xL, min(q2#xL) AS min(q2)#xL] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT max(q1), max(q2) FROM INT8_TBL +-- !query analysis +Aggregate [max(q1#xL) AS max(q1)#xL, max(q2#xL) AS max(q2)#xL] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS to_char_1, to_char(q1, '9G999G999G999G999G999'), to_char(q2, '9,999,999,999,999,999') +FROM INT8_TBL +-- !query analysis +Project [ AS to_char_1#x, to_char(cast(q1#xL as decimal(20,0)), 9G999G999G999G999G999) AS to_char(q1, 9G999G999G999G999G999)#x, to_char(cast(q2#xL as decimal(20,0)), 9,999,999,999,999,999) AS to_char(q2, 9,999,999,999,999,999)#x] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS to_char_3, to_char( (q1 * -1), '9999999999999999PR'), to_char( (q2 * -1), '9999999999999999.999PR') +FROM INT8_TBL +-- !query analysis +Project [ AS to_char_3#x, to_char(cast((q1#xL * cast(-1 as bigint)) as decimal(20,0)), 9999999999999999PR) AS to_char((q1 * -1), 9999999999999999PR)#x, to_char(cast((q2#xL * cast(-1 as bigint)) as decimal(20,0)), 9999999999999999.999PR) AS to_char((q2 * -1), 9999999999999999.999PR)#x] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS to_char_4, to_char( (q1 * -1), '9999999999999999S'), to_char( (q2 * -1), 'S9999999999999999') +FROM INT8_TBL +-- !query analysis +Project [ AS to_char_4#x, to_char(cast((q1#xL * cast(-1 as bigint)) as decimal(20,0)), 9999999999999999S) AS to_char((q1 * -1), 9999999999999999S)#x, to_char(cast((q2#xL * cast(-1 as bigint)) as decimal(20,0)), S9999999999999999) AS to_char((q2 * -1), S9999999999999999)#x] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS to_char_5, to_char(q2, 'MI9999999999999999') FROM INT8_TBL +-- !query analysis +Project [ AS to_char_5#x, to_char(cast(q2#xL as decimal(20,0)), MI9999999999999999) AS to_char(q2, MI9999999999999999)#x] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS to_char_9, to_char(q2, '0999999999999999') FROM INT8_TBL +-- !query analysis +Project [ AS to_char_9#x, to_char(cast(q2#xL as decimal(20,0)), 0999999999999999) AS to_char(q2, 0999999999999999)#x] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT '' AS to_char_10, to_char(q2, 'S0999999999999999') FROM INT8_TBL +-- !query analysis +Project [ AS to_char_10#x, to_char(cast(q2#xL as decimal(20,0)), S0999999999999999) AS to_char(q2, S0999999999999999)#x] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +select bigint('9223372036854775800') / bigint('0') +-- !query analysis +Project [(cast(cast(9223372036854775800 as bigint) as double) / cast(cast(0 as bigint) as double)) AS (9223372036854775800 / 0)#x] ++- OneRowRelation + + +-- !query +select bigint('-9223372036854775808') / smallint('0') +-- !query analysis +Project [(cast(cast(-9223372036854775808 as bigint) as double) / cast(cast(0 as smallint) as double)) AS (-9223372036854775808 / 0)#x] ++- OneRowRelation + + +-- !query +select smallint('100') / bigint('0') +-- !query analysis +Project [(cast(cast(100 as smallint) as double) / cast(cast(0 as bigint) as double)) AS (100 / 0)#x] ++- OneRowRelation + + +-- !query +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 = 456 +-- !query analysis +Project [cast(q1#xL as int) AS q1#x] ++- Filter (q2#xL = cast(456 as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 <> 456 +-- !query analysis +Project [cast(q1#xL as int) AS q1#x] ++- Filter NOT (q2#xL = cast(456 as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 = 456 +-- !query analysis +Project [cast(q1#xL as smallint) AS q1#x] ++- Filter (q2#xL = cast(456 as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 <> 456 +-- !query analysis +Project [cast(q1#xL as smallint) AS q1#x] ++- Filter NOT (q2#xL = cast(456 as bigint)) + +- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT CAST(smallint('42') AS bigint), CAST(smallint('-37') AS bigint) +-- !query analysis +Project [cast(cast(42 as smallint) as bigint) AS CAST(42 AS BIGINT)#xL, cast(cast(-37 as smallint) as bigint) AS CAST(-37 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST(q1 AS float), CAST(q2 AS double) FROM INT8_TBL +-- !query analysis +Project [cast(q1#xL as float) AS q1#x, cast(q2#xL as double) AS q2#x] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT CAST(float('36854775807.0') AS bigint) +-- !query analysis +Project [cast(cast(36854775807.0 as float) as bigint) AS CAST(36854775807.0 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT CAST(double('922337203685477580700.0') AS bigint) +-- !query analysis +Project [cast(cast(922337203685477580700.0 as double) as bigint) AS CAST(922337203685477580700.0 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT q1, q2, q1 & q2 AS `and`, q1 | q2 AS `or`, ~q1 AS `not` FROM INT8_TBL +-- !query analysis +Project [q1#xL, q2#xL, (q1#xL & q2#xL) AS and#xL, (q1#xL | q2#xL) AS or#xL, ~q1#xL AS not#xL] ++- SubqueryAlias spark_catalog.default.int8_tbl + +- Relation spark_catalog.default.int8_tbl[q1#xL,q2#xL] parquet + + +-- !query +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799')) +-- !query analysis +Project [id#xL] ++- Range (4567890123456789, 4567890123456799, step=1, splits=None) + + +-- !query +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 0) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "FAILED_FUNCTION_CALL", + "sqlState" : "38000", + "messageParameters" : { + "funcName" : "`range`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 80, + "fragment" : "range(bigint('+4567890123456789'), bigint('+4567890123456799'), 0)" + } ] +} + + +-- !query +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 2) +-- !query analysis +Project [id#xL] ++- Range (4567890123456789, 4567890123456799, step=2, splits=None) + + +-- !query +SELECT string(shiftleft(bigint(-1), 63)) +-- !query analysis +Project [cast(shiftleft(cast(-1 as bigint), 63) as string) AS shiftleft(-1, 63)#x] ++- OneRowRelation + + +-- !query +SELECT string(int(shiftleft(bigint(-1), 63))+1) +-- !query analysis +Project [cast((cast(shiftleft(cast(-1 as bigint), 63) as int) + 1) as string) AS (shiftleft(-1, 63) + 1)#x] ++- OneRowRelation + + +-- !query +SELECT bigint((-9223372036854775808)) * bigint((-1)) +-- !query analysis +Project [(cast(-9223372036854775808 as bigint) * cast(-1 as bigint)) AS (-9223372036854775808 * -1)#xL] ++- OneRowRelation + + +-- !query +SELECT bigint((-9223372036854775808)) / bigint((-1)) +-- !query analysis +Project [(cast(cast(-9223372036854775808 as bigint) as double) / cast(cast(-1 as bigint) as double)) AS (-9223372036854775808 / -1)#x] ++- OneRowRelation + + +-- !query +SELECT bigint((-9223372036854775808)) % bigint((-1)) +-- !query analysis +Project [(cast(-9223372036854775808 as bigint) % cast(-1 as bigint)) AS (-9223372036854775808 % -1)#xL] ++- OneRowRelation + + +-- !query +SELECT bigint((-9223372036854775808)) * int((-1)) +-- !query analysis +Project [(cast(-9223372036854775808 as bigint) * cast(cast(-1 as int) as bigint)) AS (-9223372036854775808 * -1)#xL] ++- OneRowRelation + + +-- !query +SELECT bigint((-9223372036854775808)) / int((-1)) +-- !query analysis +Project [(cast(cast(-9223372036854775808 as bigint) as double) / cast(cast(-1 as int) as double)) AS (-9223372036854775808 / -1)#x] ++- OneRowRelation + + +-- !query +SELECT bigint((-9223372036854775808)) % int((-1)) +-- !query analysis +Project [(cast(-9223372036854775808 as bigint) % cast(cast(-1 as int) as bigint)) AS (-9223372036854775808 % -1)#xL] ++- OneRowRelation + + +-- !query +SELECT bigint((-9223372036854775808)) * smallint((-1)) +-- !query analysis +Project [(cast(-9223372036854775808 as bigint) * cast(cast(-1 as smallint) as bigint)) AS (-9223372036854775808 * -1)#xL] ++- OneRowRelation + + +-- !query +SELECT bigint((-9223372036854775808)) / smallint((-1)) +-- !query analysis +Project [(cast(cast(-9223372036854775808 as bigint) as double) / cast(cast(-1 as smallint) as double)) AS (-9223372036854775808 / -1)#x] ++- OneRowRelation + + +-- !query +SELECT bigint((-9223372036854775808)) % smallint((-1)) +-- !query analysis +Project [(cast(-9223372036854775808 as bigint) % cast(cast(-1 as smallint) as bigint)) AS (-9223372036854775808 % -1)#xL] ++- OneRowRelation + + +-- !query +SELECT x, bigint(x) AS int8_value +FROM (VALUES (double(-2.5)), + (double(-1.5)), + (double(-0.5)), + (double(0.0)), + (double(0.5)), + (double(1.5)), + (double(2.5))) t(x) +-- !query analysis +Project [x#x, cast(x#x as bigint) AS int8_value#xL] ++- SubqueryAlias t + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT x, bigint(x) AS int8_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x) +-- !query analysis +Project [x#x, cast(x#x as bigint) AS int8_value#xL] ++- SubqueryAlias t + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +DROP TABLE INT8_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`INT8_TBL`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/interval.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/interval.sql.out new file mode 100644 index 0000000000000..8d41651cb743a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/interval.sql.out @@ -0,0 +1,287 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT interval '999' second +-- !query analysis +Project [16 minutes 39 seconds AS INTERVAL '16 minutes 39 seconds'#x] ++- OneRowRelation + + +-- !query +SELECT interval '999' minute +-- !query analysis +Project [16 hours 39 minutes AS INTERVAL '16 hours 39 minutes'#x] ++- OneRowRelation + + +-- !query +SELECT interval '999' hour +-- !query analysis +Project [999 hours AS INTERVAL '999 hours'#x] ++- OneRowRelation + + +-- !query +SELECT interval '999' day +-- !query analysis +Project [999 days AS INTERVAL '999 days'#x] ++- OneRowRelation + + +-- !query +SELECT interval '999' month +-- !query analysis +Project [83 years 3 months AS INTERVAL '83 years 3 months'#x] ++- OneRowRelation + + +-- !query +SELECT interval '1' year +-- !query analysis +Project [1 years AS INTERVAL '1 years'#x] ++- OneRowRelation + + +-- !query +SELECT interval '2' month +-- !query analysis +Project [2 months AS INTERVAL '2 months'#x] ++- OneRowRelation + + +-- !query +SELECT interval '3' day +-- !query analysis +Project [3 days AS INTERVAL '3 days'#x] ++- OneRowRelation + + +-- !query +SELECT interval '4' hour +-- !query analysis +Project [4 hours AS INTERVAL '4 hours'#x] ++- OneRowRelation + + +-- !query +SELECT interval '5' minute +-- !query analysis +Project [5 minutes AS INTERVAL '5 minutes'#x] ++- OneRowRelation + + +-- !query +SELECT interval '6' second +-- !query analysis +Project [6 seconds AS INTERVAL '6 seconds'#x] ++- OneRowRelation + + +-- !query +SELECT interval '1-2' year to month +-- !query analysis +Project [1 years 2 months AS INTERVAL '1 years 2 months'#x] ++- OneRowRelation + + +-- !query +SELECT interval '1 2:03' day to hour +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 36, + "fragment" : "'1 2:03' day to hour" + } ] +} + + +-- !query +SELECT interval '1 2:03:04' day to hour +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 39, + "fragment" : "'1 2:03:04' day to hour" + } ] +} + + +-- !query +SELECT interval '1 2:03' day to minute +-- !query analysis +Project [1 days 2 hours 3 minutes AS INTERVAL '1 days 2 hours 3 minutes'#x] ++- OneRowRelation + + +-- !query +SELECT interval '1 2:03:04' day to minute +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE` when cast to interval day to minute: 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 41, + "fragment" : "'1 2:03:04' day to minute" + } ] +} + + +-- !query +SELECT interval '1 2:03' day to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND` when cast to interval day to second: 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 38, + "fragment" : "'1 2:03' day to second" + } ] +} + + +-- !query +SELECT interval '1 2:03:04' day to second +-- !query analysis +Project [1 days 2 hours 3 minutes 4 seconds AS INTERVAL '1 days 2 hours 3 minutes 4 seconds'#x] ++- OneRowRelation + + +-- !query +SELECT interval '1 2:03' hour to minute +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 39, + "fragment" : "'1 2:03' hour to minute" + } ] +} + + +-- !query +SELECT interval '1 2:03:04' hour to minute +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 42, + "fragment" : "'1 2:03:04' hour to minute" + } ] +} + + +-- !query +SELECT interval '1 2:03' hour to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 39, + "fragment" : "'1 2:03' hour to second" + } ] +} + + +-- !query +SELECT interval '1 2:03:04' hour to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 42, + "fragment" : "'1 2:03:04' hour to second" + } ] +} + + +-- !query +SELECT interval '1 2:03' minute to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 41, + "fragment" : "'1 2:03' minute to second" + } ] +} + + +-- !query +SELECT interval '1 2:03:04' minute to second +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 44, + "fragment" : "'1 2:03:04' minute to second" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out new file mode 100644 index 0000000000000..f05af60b25461 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out @@ -0,0 +1,3567 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT2_TBL(f1) AS VALUES + (smallint(trim('0 '))), + (smallint(trim(' 1234 '))), + (smallint(trim(' -1234'))), + (smallint('32767')), + (smallint('-32767')) +-- !query analysis +CreateViewCommand `INT2_TBL`, [(f1,None)], VALUES + (smallint(trim('0 '))), + (smallint(trim(' 1234 '))), + (smallint(trim(' -1234'))), + (smallint('32767')), + (smallint('-32767')), false, true, LocalTempView, true + +- LocalRelation [col1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM + (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) + AS v(f1) +-- !query analysis +CreateViewCommand `INT4_TBL`, SELECT * FROM + (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) + AS v(f1), false, true, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2) +-- !query analysis +CreateViewCommand `INT8_TBL`, SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2), false, true, LocalTempView, true + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM + (VALUES (0.0), (1004.30), (-34.84), + (cast('1.2345678901234e+200' as double)), (cast('1.2345678901234e-200' as double))) + AS v(f1) +-- !query analysis +CreateViewCommand `FLOAT8_TBL`, SELECT * FROM + (VALUES (0.0), (1004.30), (-34.84), + (cast('1.2345678901234e+200' as double)), (cast('1.2345678901234e-200' as double))) + AS v(f1), false, true, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW TEXT_TBL AS SELECT * FROM + (VALUES ('doh!'), ('hi de ho neighbor')) + AS v(f1) +-- !query analysis +CreateViewCommand `TEXT_TBL`, SELECT * FROM + (VALUES ('doh!'), ('hi de ho neighbor')) + AS v(f1), false, true, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 +-- !query analysis +CreateViewCommand `tenk2`, SELECT * FROM tenk1, false, true, LocalTempView, true + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +CREATE TABLE J1_TBL ( + i integer, + j integer, + t string +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`J1_TBL`, false + + +-- !query +CREATE TABLE J2_TBL ( + i integer, + k integer +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`J2_TBL`, false + + +-- !query +INSERT INTO J1_TBL VALUES (1, 4, 'one') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (2, 3, 'two') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (3, 2, 'three') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (4, 1, 'four') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (5, 0, 'five') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (6, 6, 'six') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (7, 7, 'seven') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (8, 8, 'eight') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (0, NULL, 'zero') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (NULL, NULL, 'null') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (NULL, 0, 'zero') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J2_TBL VALUES (1, -1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (2, 2) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (3, -3) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (2, 4) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (5, -5) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (5, -5) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (0, NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (NULL, NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (NULL, 0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL AS tx +-- !query analysis +Project [ AS xxx#x, i#x, j#x, t#x] ++- SubqueryAlias tx + +- SubqueryAlias spark_catalog.default.j1_tbl + +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL tx +-- !query analysis +Project [ AS xxx#x, i#x, j#x, t#x] ++- SubqueryAlias tx + +- SubqueryAlias spark_catalog.default.j1_tbl + +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL AS t1 (a, b, c) +-- !query analysis +Project [ AS xxx#x, a#x, b#x, c#x] ++- SubqueryAlias t1 + +- Project [i#x AS a#x, j#x AS b#x, t#x AS c#x] + +- SubqueryAlias spark_catalog.default.j1_tbl + +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL t1 (a, b, c) +-- !query analysis +Project [ AS xxx#x, a#x, b#x, c#x] ++- SubqueryAlias t1 + +- Project [i#x AS a#x, j#x AS b#x, t#x AS c#x] + +- SubqueryAlias spark_catalog.default.j1_tbl + +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e) +-- !query analysis +Project [ AS xxx#x, a#x, b#x, c#x, d#x, e#x] ++- Join Inner + :- SubqueryAlias t1 + : +- Project [i#x AS a#x, j#x AS b#x, t#x AS c#x] + : +- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias t2 + +- Project [i#x AS d#x, k#x AS e#x] + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL CROSS JOIN J2_TBL +-- !query analysis +Project [ AS xxx#x, i#x, j#x, t#x, i#x, k#x] ++- Join Cross + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, i, k, t + FROM J1_TBL CROSS JOIN J2_TBL +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`i`", + "referenceNames" : "[`spark_catalog`.`default`.`j1_tbl`.`i`, `spark_catalog`.`default`.`j2_tbl`.`i`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 21, + "stopIndex" : 21, + "fragment" : "i" + } ] +} + + +-- !query +SELECT '' AS `xxx`, t1.i, k, t + FROM J1_TBL t1 CROSS JOIN J2_TBL t2 +-- !query analysis +Project [ AS xxx#x, i#x, k#x, t#x] ++- Join Cross + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, ii, tt, kk + FROM (J1_TBL CROSS JOIN J2_TBL) + AS tx (ii, jj, tt, ii2, kk) +-- !query analysis +Project [ AS xxx#x, ii#x, tt#x, kk#x] ++- SubqueryAlias tx + +- Project [i#x AS ii#x, j#x AS jj#x, t#x AS tt#x, i#x AS ii2#x, k#x AS kk#x] + +- Join Cross + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL CROSS JOIN J2_TBL a CROSS JOIN J2_TBL b +-- !query analysis +Project [ AS xxx#x, i#x, j#x, t#x, i#x, k#x, i#x, k#x] ++- Join Cross + :- Join Cross + : :- SubqueryAlias spark_catalog.default.j1_tbl + : : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + : +- SubqueryAlias a + : +- SubqueryAlias spark_catalog.default.j2_tbl + : +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL INNER JOIN J2_TBL USING (i) +-- !query analysis +Project [ AS xxx#x, i#x, j#x, t#x, k#x] ++- Project [i#x, j#x, t#x, k#x] + +- Join Inner, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL JOIN J2_TBL USING (i) +-- !query analysis +Project [ AS xxx#x, i#x, j#x, t#x, k#x] ++- Project [i#x, j#x, t#x, k#x] + +- Join Inner, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, d) USING (a) + ORDER BY a, d +-- !query analysis +Sort [a#x ASC NULLS FIRST, d#x ASC NULLS FIRST], true ++- Project [ AS xxx#x, a#x, b#x, c#x, d#x] + +- Project [a#x, b#x, c#x, d#x] + +- Join Inner, (a#x = a#x) + :- SubqueryAlias t1 + : +- Project [i#x AS a#x, j#x AS b#x, t#x AS c#x] + : +- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias t2 + +- Project [i#x AS a#x, k#x AS d#x] + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL NATURAL JOIN J2_TBL +-- !query analysis +Project [ AS xxx#x, i#x, j#x, t#x, k#x] ++- Project [i#x, j#x, t#x, k#x] + +- Join Inner, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (a, d) +-- !query analysis +Project [ AS xxx#x, a#x, b#x, c#x, d#x] ++- Project [a#x, b#x, c#x, d#x] + +- Join Inner, (a#x = a#x) + :- SubqueryAlias t1 + : +- Project [i#x AS a#x, j#x AS b#x, t#x AS c#x] + : +- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias t2 + +- Project [i#x AS a#x, k#x AS d#x] + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (d, a) +-- !query analysis +Project [ AS xxx#x, a#x, b#x, c#x, d#x] ++- Project [a#x, b#x, c#x, d#x] + +- Join Inner, (a#x = a#x) + :- SubqueryAlias t1 + : +- Project [i#x AS a#x, j#x AS b#x, t#x AS c#x] + : +- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias t2 + +- Project [i#x AS d#x, k#x AS a#x] + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = J2_TBL.i) +-- !query analysis +Project [ AS xxx#x, i#x, j#x, t#x, i#x, k#x] ++- Join Inner, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = J2_TBL.k) +-- !query analysis +Project [ AS xxx#x, i#x, j#x, t#x, i#x, k#x] ++- Join Inner, (i#x = k#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i <= J2_TBL.k) +-- !query analysis +Project [ AS xxx#x, i#x, j#x, t#x, i#x, k#x] ++- Join Inner, (i#x <= k#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL LEFT OUTER JOIN J2_TBL USING (i) + ORDER BY i, k, t +-- !query analysis +Sort [i#x ASC NULLS FIRST, k#x ASC NULLS FIRST, t#x ASC NULLS FIRST], true ++- Project [ AS xxx#x, i#x, j#x, t#x, k#x] + +- Project [i#x, j#x, t#x, k#x] + +- Join LeftOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL LEFT JOIN J2_TBL USING (i) + ORDER BY i, k, t +-- !query analysis +Sort [i#x ASC NULLS FIRST, k#x ASC NULLS FIRST, t#x ASC NULLS FIRST], true ++- Project [ AS xxx#x, i#x, j#x, t#x, k#x] + +- Project [i#x, j#x, t#x, k#x] + +- Join LeftOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL RIGHT OUTER JOIN J2_TBL USING (i) +-- !query analysis +Project [ AS xxx#x, i#x, j#x, t#x, k#x] ++- Project [i#x, j#x, t#x, k#x] + +- Join RightOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL RIGHT JOIN J2_TBL USING (i) +-- !query analysis +Project [ AS xxx#x, i#x, j#x, t#x, k#x] ++- Project [i#x, j#x, t#x, k#x] + +- Join RightOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL FULL OUTER JOIN J2_TBL USING (i) + ORDER BY i, k, t +-- !query analysis +Sort [i#x ASC NULLS FIRST, k#x ASC NULLS FIRST, t#x ASC NULLS FIRST], true ++- Project [ AS xxx#x, i#x, j#x, t#x, k#x] + +- Project [coalesce(i#x, i#x) AS i#x, j#x, t#x, k#x] + +- Join FullOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL FULL JOIN J2_TBL USING (i) + ORDER BY i, k, t +-- !query analysis +Sort [i#x ASC NULLS FIRST, k#x ASC NULLS FIRST, t#x ASC NULLS FIRST], true ++- Project [ AS xxx#x, i#x, j#x, t#x, k#x] + +- Project [coalesce(i#x, i#x) AS i#x, j#x, t#x, k#x] + +- Join FullOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (k = 1) +-- !query analysis +Project [ AS xxx#x, i#x, j#x, t#x, k#x] ++- Filter (k#x = 1) + +- Project [i#x, j#x, t#x, k#x] + +- Join LeftOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (i = 1) +-- !query analysis +Project [ AS xxx#x, i#x, j#x, t#x, k#x] ++- Filter (i#x = 1) + +- Project [i#x, j#x, t#x, k#x] + +- Join LeftOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +CREATE TABLE t1 (name STRING, n INTEGER) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false + + +-- !query +CREATE TABLE t2 (name STRING, n INTEGER) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t2`, false + + +-- !query +CREATE TABLE t3 (name STRING, n INTEGER) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t3`, false + + +-- !query +INSERT INTO t1 VALUES ( 'bb', 11 ) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO t2 VALUES ( 'bb', 12 ) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO t2 VALUES ( 'cc', 22 ) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO t2 VALUES ( 'ee', 42 ) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO t3 VALUES ( 'bb', 13 ) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t3], Append, `spark_catalog`.`default`.`t3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t3), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO t3 VALUES ( 'cc', 23 ) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t3], Append, `spark_catalog`.`default`.`t3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t3), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO t3 VALUES ( 'dd', 33 ) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t3], Append, `spark_catalog`.`default`.`t3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t3), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name) +-- !query analysis +Project [name#x, n#x, n#x, n#x] ++- Project [coalesce(name#x, name#x) AS name#x, n#x, n#x, n#x] + +- Join FullOuter, (name#x = name#x) + :- Project [coalesce(name#x, name#x) AS name#x, n#x, n#x] + : +- Join FullOuter, (name#x = name#x) + : :- SubqueryAlias spark_catalog.default.t1 + : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT * FROM t2) as s2 +INNER JOIN +(SELECT * FROM t3) s3 +USING (name) +-- !query analysis +Project [name#x, n#x, n#x] ++- Project [name#x, n#x, n#x] + +- Join Inner, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [name#x, n#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, n#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT * FROM t2) as s2 +LEFT JOIN +(SELECT * FROM t3) s3 +USING (name) +-- !query analysis +Project [name#x, n#x, n#x] ++- Project [name#x, n#x, n#x] + +- Join LeftOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [name#x, n#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, n#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT * FROM t2) as s2 +FULL JOIN +(SELECT * FROM t3) s3 +USING (name) +-- !query analysis +Project [name#x, n#x, n#x] ++- Project [coalesce(name#x, name#x) AS name#x, n#x, n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [name#x, n#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, n#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 +-- !query analysis +Project [name#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] ++- Project [name#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] + +- Join Inner, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [name#x, n#x AS s2_n#x, 2 AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, n#x AS s3_n#x, 3 AS s3_2#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL LEFT JOIN +(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 +-- !query analysis +Project [name#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] ++- Project [name#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] + +- Join LeftOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [name#x, n#x AS s2_n#x, 2 AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, n#x AS s3_n#x, 3 AS s3_2#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 +-- !query analysis +Project [name#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] ++- Project [coalesce(name#x, name#x) AS name#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [name#x, n#x AS s2_n#x, 2 AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, n#x AS s3_n#x, 3 AS s3_2#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT name, n as s1_n, 1 as s1_1 FROM t1) as s1 +NATURAL INNER JOIN +(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 +-- !query analysis +Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] ++- Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] + +- Join Inner, (name#x = name#x) + :- Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x] + : +- Join Inner, (name#x = name#x) + : :- SubqueryAlias s1 + : : +- Project [name#x, n#x AS s1_n#x, 1 AS s1_1#x] + : : +- SubqueryAlias spark_catalog.default.t1 + : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + : +- SubqueryAlias s2 + : +- Project [name#x, n#x AS s2_n#x, 2 AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, n#x AS s3_n#x, 3 AS s3_2#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT name, n as s1_n, 1 as s1_1 FROM t1) as s1 +NATURAL FULL JOIN +(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 +-- !query analysis +Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] ++- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] + +- Join FullOuter, (name#x = name#x) + :- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x] + : +- Join FullOuter, (name#x = name#x) + : :- SubqueryAlias s1 + : : +- Project [name#x, n#x AS s1_n#x, 1 AS s1_1#x] + : : +- SubqueryAlias spark_catalog.default.t1 + : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + : +- SubqueryAlias s2 + : +- Project [name#x, n#x AS s2_n#x, 2 AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, n#x AS s3_n#x, 3 AS s3_2#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT name, n as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, n as s2_n FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, n as s3_n FROM t3) as s3 + ) ss2 +-- !query analysis +Project [name#x, s1_n#x, s2_n#x, s3_n#x] ++- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s2_n#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s1 + : +- Project [name#x, n#x AS s1_n#x] + : +- SubqueryAlias spark_catalog.default.t1 + : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + +- SubqueryAlias ss2 + +- Project [name#x, s2_n#x, s3_n#x] + +- Project [coalesce(name#x, name#x) AS name#x, s2_n#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [name#x, n#x AS s2_n#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, n#x AS s3_n#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT name, n as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, n as s3_n FROM t3) as s3 + ) ss2 +-- !query analysis +Project [name#x, s1_n#x, s2_n#x, s2_2#x, s3_n#x] ++- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s2_n#x, s2_2#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s1 + : +- Project [name#x, n#x AS s1_n#x] + : +- SubqueryAlias spark_catalog.default.t1 + : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + +- SubqueryAlias ss2 + +- Project [name#x, s2_n#x, s2_2#x, s3_n#x] + +- Project [coalesce(name#x, name#x) AS name#x, s2_n#x, s2_2#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [name#x, n#x AS s2_n#x, 2 AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, n#x AS s3_n#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM + (SELECT name, n as s1_n FROM t1) as s1 +FULL JOIN + (SELECT name, 2 as s2_n FROM t2) as s2 +ON (s1_n = s2_n) +-- !query analysis +Project [name#x, s1_n#x, name#x, s2_n#x] ++- Join FullOuter, (s1_n#x = s2_n#x) + :- SubqueryAlias s1 + : +- Project [name#x, n#x AS s1_n#x] + : +- SubqueryAlias spark_catalog.default.t1 + : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + +- SubqueryAlias s2 + +- Project [name#x, 2 AS s2_n#x] + +- SubqueryAlias spark_catalog.default.t2 + +- Relation spark_catalog.default.t2[name#x,n#x] parquet + + +-- !query +create or replace temporary view x as select * from + (values (1,11), (2,22), (3,null), (4,44), (5,null)) + as v(x1, x2) +-- !query analysis +CreateViewCommand `x`, select * from + (values (1,11), (2,22), (3,null), (4,44), (5,null)) + as v(x1, x2), false, true, LocalTempView, true + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +create or replace temporary view y as select * from + (values (1,111), (2,222), (3,333), (4,null)) + as v(y1, y2) +-- !query analysis +CreateViewCommand `y`, select * from + (values (1,111), (2,222), (3,333), (4,null)) + as v(y1, y2), false, true, LocalTempView, true + +- Project [y1#x, y2#x] + +- SubqueryAlias v + +- Project [col1#x AS y1#x, col2#x AS y2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from x +-- !query analysis +Project [x1#x, x2#x] ++- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from y +-- !query analysis +Project [y1#x, y2#x] ++- SubqueryAlias y + +- View (`y`, [y1#x,y2#x]) + +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + +- Project [y1#x, y2#x] + +- SubqueryAlias v + +- Project [col1#x AS y1#x, col2#x AS y2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from x left join y on (x1 = y1 and x2 is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x] ++- Join LeftOuter, ((x1#x = y1#x) AND isnotnull(x2#x)) + :- SubqueryAlias x + : +- View (`x`, [x1#x,x2#x]) + : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : +- Project [x1#x, x2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias y + +- View (`y`, [y1#x,y2#x]) + +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + +- Project [y1#x, y2#x] + +- SubqueryAlias v + +- Project [col1#x AS y1#x, col2#x AS y2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from x left join y on (x1 = y1 and y2 is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x] ++- Join LeftOuter, ((x1#x = y1#x) AND isnotnull(y2#x)) + :- SubqueryAlias x + : +- View (`x`, [x1#x,x2#x]) + : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : +- Project [x1#x, x2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias y + +- View (`y`, [y1#x,y2#x]) + +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + +- Project [y1#x, y2#x] + +- SubqueryAlias v + +- Project [col1#x AS y1#x, col2#x AS y2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x, xx1#x, xx2#x] ++- Join LeftOuter, (x1#x = xx1#x) + :- Join LeftOuter, (x1#x = y1#x) + : :- SubqueryAlias x + : : +- View (`x`, [x1#x,x2#x]) + : : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : : +- Project [x1#x, x2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias y + : +- View (`y`, [y1#x,y2#x]) + : +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + : +- Project [y1#x, y2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS y1#x, col2#x AS y2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias xx + +- Project [x1#x AS xx1#x, x2#x AS xx2#x] + +- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1 and x2 is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x, xx1#x, xx2#x] ++- Join LeftOuter, ((x1#x = xx1#x) AND isnotnull(x2#x)) + :- Join LeftOuter, (x1#x = y1#x) + : :- SubqueryAlias x + : : +- View (`x`, [x1#x,x2#x]) + : : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : : +- Project [x1#x, x2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias y + : +- View (`y`, [y1#x,y2#x]) + : +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + : +- Project [y1#x, y2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS y1#x, col2#x AS y2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias xx + +- Project [x1#x AS xx1#x, x2#x AS xx2#x] + +- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1 and y2 is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x, xx1#x, xx2#x] ++- Join LeftOuter, ((x1#x = xx1#x) AND isnotnull(y2#x)) + :- Join LeftOuter, (x1#x = y1#x) + : :- SubqueryAlias x + : : +- View (`x`, [x1#x,x2#x]) + : : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : : +- Project [x1#x, x2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias y + : +- View (`y`, [y1#x,y2#x]) + : +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + : +- Project [y1#x, y2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS y1#x, col2#x AS y2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias xx + +- Project [x1#x AS xx1#x, x2#x AS xx2#x] + +- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1 and xx2 is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x, xx1#x, xx2#x] ++- Join LeftOuter, ((x1#x = xx1#x) AND isnotnull(xx2#x)) + :- Join LeftOuter, (x1#x = y1#x) + : :- SubqueryAlias x + : : +- View (`x`, [x1#x,x2#x]) + : : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : : +- Project [x1#x, x2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias y + : +- View (`y`, [y1#x,y2#x]) + : +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + : +- Project [y1#x, y2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS y1#x, col2#x AS y2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias xx + +- Project [x1#x AS xx1#x, x2#x AS xx2#x] + +- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1) where (x2 is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x, xx1#x, xx2#x] ++- Filter isnotnull(x2#x) + +- Join LeftOuter, (x1#x = xx1#x) + :- Join LeftOuter, (x1#x = y1#x) + : :- SubqueryAlias x + : : +- View (`x`, [x1#x,x2#x]) + : : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : : +- Project [x1#x, x2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias y + : +- View (`y`, [y1#x,y2#x]) + : +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + : +- Project [y1#x, y2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS y1#x, col2#x AS y2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias xx + +- Project [x1#x AS xx1#x, x2#x AS xx2#x] + +- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1) where (y2 is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x, xx1#x, xx2#x] ++- Filter isnotnull(y2#x) + +- Join LeftOuter, (x1#x = xx1#x) + :- Join LeftOuter, (x1#x = y1#x) + : :- SubqueryAlias x + : : +- View (`x`, [x1#x,x2#x]) + : : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : : +- Project [x1#x, x2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias y + : +- View (`y`, [y1#x,y2#x]) + : +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + : +- Project [y1#x, y2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS y1#x, col2#x AS y2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias xx + +- Project [x1#x AS xx1#x, x2#x AS xx2#x] + +- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1) where (xx2 is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x, xx1#x, xx2#x] ++- Filter isnotnull(xx2#x) + +- Join LeftOuter, (x1#x = xx1#x) + :- Join LeftOuter, (x1#x = y1#x) + : :- SubqueryAlias x + : : +- View (`x`, [x1#x,x2#x]) + : : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : : +- Project [x1#x, x2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias y + : +- View (`y`, [y1#x,y2#x]) + : +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + : +- Project [y1#x, y2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS y1#x, col2#x AS y2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias xx + +- Project [x1#x AS xx1#x, x2#x AS xx2#x] + +- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select count(*) from tenk1 a where unique1 in + (select unique1 from tenk1 b join tenk1 c using (unique1) + where b.unique2 = 42) +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter unique1#x IN (list#x []) + : +- Project [unique1#x] + : +- Filter (unique2#x = 42) + : +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, ... 7 more fields] + : +- Join Inner, (unique1#x = unique1#x) + : :- SubqueryAlias b + : : +- SubqueryAlias spark_catalog.default.tenk1 + : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : +- SubqueryAlias c + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias a + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select count(*) from tenk1 x where + x.unique1 in (select a.f1 from int4_tbl a,float8_tbl b where a.f1=b.f1) and + x.unique1 = 0 and + x.unique1 in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=bb.f1) +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter ((unique1#x IN (list#x []) AND (unique1#x = 0)) AND unique1#x IN (list#x [])) + : :- Project [f1#x] + : : +- Filter (cast(f1#x as double) = f1#x) + : : +- Join Inner + : : :- SubqueryAlias a + : : : +- SubqueryAlias int4_tbl + : : : +- View (`INT4_TBL`, [f1#x]) + : : : +- Project [cast(f1#x as int) AS f1#x] + : : : +- Project [f1#x] + : : : +- SubqueryAlias v + : : : +- Project [col1#x AS f1#x] + : : : +- LocalRelation [col1#x] + : : +- SubqueryAlias b + : : +- SubqueryAlias float8_tbl + : : +- View (`FLOAT8_TBL`, [f1#x]) + : : +- Project [cast(f1#x as double) AS f1#x] + : : +- Project [f1#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS f1#x] + : : +- LocalRelation [col1#x] + : +- Project [f1#x] + : +- Filter (cast(f1#x as double) = f1#x) + : +- Join Inner + : :- SubqueryAlias aa + : : +- SubqueryAlias int4_tbl + : : +- View (`INT4_TBL`, [f1#x]) + : : +- Project [cast(f1#x as int) AS f1#x] + : : +- Project [f1#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS f1#x] + : : +- LocalRelation [col1#x] + : +- SubqueryAlias bb + : +- SubqueryAlias float8_tbl + : +- View (`FLOAT8_TBL`, [f1#x]) + : +- Project [cast(f1#x as double) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias x + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select count(*) from tenk1 x where + x.unique1 in (select a.f1 from int4_tbl a,float8_tbl b where a.f1=b.f1) and + x.unique1 = 0 and + x.unique1 in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=bb.f1) +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter ((unique1#x IN (list#x []) AND (unique1#x = 0)) AND unique1#x IN (list#x [])) + : :- Project [f1#x] + : : +- Filter (cast(f1#x as double) = f1#x) + : : +- Join Inner + : : :- SubqueryAlias a + : : : +- SubqueryAlias int4_tbl + : : : +- View (`INT4_TBL`, [f1#x]) + : : : +- Project [cast(f1#x as int) AS f1#x] + : : : +- Project [f1#x] + : : : +- SubqueryAlias v + : : : +- Project [col1#x AS f1#x] + : : : +- LocalRelation [col1#x] + : : +- SubqueryAlias b + : : +- SubqueryAlias float8_tbl + : : +- View (`FLOAT8_TBL`, [f1#x]) + : : +- Project [cast(f1#x as double) AS f1#x] + : : +- Project [f1#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS f1#x] + : : +- LocalRelation [col1#x] + : +- Project [f1#x] + : +- Filter (cast(f1#x as double) = f1#x) + : +- Join Inner + : :- SubqueryAlias aa + : : +- SubqueryAlias int4_tbl + : : +- View (`INT4_TBL`, [f1#x]) + : : +- Project [cast(f1#x as int) AS f1#x] + : : +- Project [f1#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS f1#x] + : : +- LocalRelation [col1#x] + : +- SubqueryAlias bb + : +- SubqueryAlias float8_tbl + : +- View (`FLOAT8_TBL`, [f1#x]) + : +- Project [cast(f1#x as double) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias x + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select * from int8_tbl i1 left join (int8_tbl i2 join + (select 123 as x) ss on i2.q1 = x) on i1.q2 = i2.q2 +order by 1, 2 +-- !query analysis +Sort [q1#xL ASC NULLS FIRST, q2#xL ASC NULLS FIRST], true ++- Project [q1#xL, q2#xL, q1#xL, q2#xL, x#x] + +- Join LeftOuter, (q2#xL = q2#xL) + :- SubqueryAlias i1 + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Join Inner, (q1#xL = cast(x#x as bigint)) + :- SubqueryAlias i2 + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias ss + +- Project [123 AS x#x] + +- OneRowRelation + + +-- !query +select count(*) +from + (select t3.tenthous as x1, coalesce(t1.stringu1, t2.stringu1) as x2 + from tenk1 t1 + left join tenk1 t2 on t1.unique1 = t2.unique1 + join tenk1 t3 on t1.unique2 = t3.unique2) ss, + tenk1 t4, + tenk1 t5 +where t4.thousand = t5.unique1 and ss.x1 = t4.tenthous and ss.x2 = t5.stringu1 +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (((thousand#x = unique1#x) AND (x1#x = tenthous#x)) AND (x2#x = stringu1#x)) + +- Join Inner + :- Join Inner + : :- SubqueryAlias ss + : : +- Project [tenthous#x AS x1#x, coalesce(stringu1#x, stringu1#x) AS x2#x] + : : +- Join Inner, (unique2#x = unique2#x) + : : :- Join LeftOuter, (unique1#x = unique1#x) + : : : :- SubqueryAlias t1 + : : : : +- SubqueryAlias spark_catalog.default.tenk1 + : : : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : : : +- SubqueryAlias t2 + : : : +- SubqueryAlias spark_catalog.default.tenk1 + : : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : : +- SubqueryAlias t3 + : : +- SubqueryAlias spark_catalog.default.tenk1 + : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : +- SubqueryAlias t4 + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias t5 + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select a.f1, b.f1, t.thousand, t.tenthous from + tenk1 t, + (select sum(f1)+1 as f1 from int4_tbl i4a) a, + (select sum(f1) as f1 from int4_tbl i4b) b +where b.f1 = t.thousand and a.f1 = b.f1 and (a.f1+b.f1+999) = t.tenthous +-- !query analysis +Project [f1#xL, f1#xL, thousand#x, tenthous#x] ++- Filter (((f1#xL = cast(thousand#x as bigint)) AND (f1#xL = f1#xL)) AND (((f1#xL + f1#xL) + cast(999 as bigint)) = cast(tenthous#x as bigint))) + +- Join Inner + :- Join Inner + : :- SubqueryAlias t + : : +- SubqueryAlias spark_catalog.default.tenk1 + : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : +- SubqueryAlias a + : +- Aggregate [(sum(f1#x) + cast(1 as bigint)) AS f1#xL] + : +- SubqueryAlias i4a + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias b + +- Aggregate [sum(f1#x) AS f1#xL] + +- SubqueryAlias i4b + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from + j1_tbl full join + (select * from j2_tbl order by j2_tbl.i desc, j2_tbl.k asc) j2_tbl + on j1_tbl.i = j2_tbl.i and j1_tbl.i = j2_tbl.k +-- !query analysis +Project [i#x, j#x, t#x, i#x, k#x] ++- Join FullOuter, ((i#x = i#x) AND (i#x = k#x)) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias j2_tbl + +- Sort [i#x DESC NULLS LAST, k#x ASC NULLS FIRST], true + +- Project [i#x, k#x] + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +select count(*) from + (select * from tenk1 x order by x.thousand, x.twothousand, x.fivethous) x + left join + (select * from tenk1 y order by y.unique2) y + on x.thousand = y.unique2 and x.twothousand = y.hundred and x.fivethous = y.unique2 +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Join LeftOuter, (((thousand#x = unique2#x) AND (twothousand#x = hundred#x)) AND (fivethous#x = unique2#x)) + :- SubqueryAlias x + : +- Sort [thousand#x ASC NULLS FIRST, twothousand#x ASC NULLS FIRST, fivethous#x ASC NULLS FIRST], true + : +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + : +- SubqueryAlias x + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias y + +- Sort [unique2#x ASC NULLS FIRST], true + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias y + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +DROP TABLE t1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t1`, false, false, false + + +-- !query +DROP TABLE t2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t2`, false, false, false + + +-- !query +DROP TABLE t3 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t3`, false, false, false + + +-- !query +DROP TABLE J1_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`J1_TBL`, false, false, false + + +-- !query +DROP TABLE J2_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`J2_TBL`, false, false, false + + +-- !query +create or replace temporary view tt1 as select * from + (values (1, 11), (2, NULL)) + as v(tt1_id, joincol) +-- !query analysis +CreateViewCommand `tt1`, select * from + (values (1, 11), (2, NULL)) + as v(tt1_id, joincol), false, true, LocalTempView, true + +- Project [tt1_id#x, joincol#x] + +- SubqueryAlias v + +- Project [col1#x AS tt1_id#x, col2#x AS joincol#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +create or replace temporary view tt2 as select * from + (values (21, 11), (22, 11)) + as v(tt2_id, joincol) +-- !query analysis +CreateViewCommand `tt2`, select * from + (values (21, 11), (22, 11)) + as v(tt2_id, joincol), false, true, LocalTempView, true + +- Project [tt2_id#x, joincol#x] + +- SubqueryAlias v + +- Project [col1#x AS tt2_id#x, col2#x AS joincol#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select tt1.*, tt2.* from tt1 left join tt2 on tt1.joincol = tt2.joincol +-- !query analysis +Project [tt1_id#x, joincol#x, tt2_id#x, joincol#x] ++- Join LeftOuter, (joincol#x = joincol#x) + :- SubqueryAlias tt1 + : +- View (`tt1`, [tt1_id#x,joincol#x]) + : +- Project [cast(tt1_id#x as int) AS tt1_id#x, cast(joincol#x as int) AS joincol#x] + : +- Project [tt1_id#x, joincol#x] + : +- SubqueryAlias v + : +- Project [col1#x AS tt1_id#x, col2#x AS joincol#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias tt2 + +- View (`tt2`, [tt2_id#x,joincol#x]) + +- Project [cast(tt2_id#x as int) AS tt2_id#x, cast(joincol#x as int) AS joincol#x] + +- Project [tt2_id#x, joincol#x] + +- SubqueryAlias v + +- Project [col1#x AS tt2_id#x, col2#x AS joincol#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select tt1.*, tt2.* from tt2 right join tt1 on tt1.joincol = tt2.joincol +-- !query analysis +Project [tt1_id#x, joincol#x, tt2_id#x, joincol#x] ++- Join RightOuter, (joincol#x = joincol#x) + :- SubqueryAlias tt2 + : +- View (`tt2`, [tt2_id#x,joincol#x]) + : +- Project [cast(tt2_id#x as int) AS tt2_id#x, cast(joincol#x as int) AS joincol#x] + : +- Project [tt2_id#x, joincol#x] + : +- SubqueryAlias v + : +- Project [col1#x AS tt2_id#x, col2#x AS joincol#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias tt1 + +- View (`tt1`, [tt1_id#x,joincol#x]) + +- Project [cast(tt1_id#x as int) AS tt1_id#x, cast(joincol#x as int) AS joincol#x] + +- Project [tt1_id#x, joincol#x] + +- SubqueryAlias v + +- Project [col1#x AS tt1_id#x, col2#x AS joincol#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select count(*) from tenk1 a, tenk1 b + where a.hundred = b.thousand and (b.fivethous % 10) < 10 +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter ((hundred#x = thousand#x) AND ((fivethous#x % 10) < 10)) + +- Join Inner + :- SubqueryAlias a + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +create or replace temporary view tt3 as select * from + (SELECT cast(x.id as int), repeat('xyzzy', 100) FROM range(1,10001) x) + as v(f1, f2) +-- !query analysis +CreateViewCommand `tt3`, select * from + (SELECT cast(x.id as int), repeat('xyzzy', 100) FROM range(1,10001) x) + as v(f1, f2), false, true, LocalTempView, true + +- Project [f1#x, f2#x] + +- SubqueryAlias v + +- Project [id#x AS f1#x, repeat(xyzzy, 100)#x AS f2#x] + +- Project [cast(id#xL as int) AS id#x, repeat(xyzzy, 100) AS repeat(xyzzy, 100)#x] + +- SubqueryAlias x + +- Range (1, 10001, step=1, splits=None) + + +-- !query +create or replace temporary view tt4 as select * from + (values (0), (1), (9999)) + as v(f1) +-- !query analysis +CreateViewCommand `tt4`, select * from + (values (0), (1), (9999)) + as v(f1), false, true, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT a.f1 +FROM tt4 a +LEFT JOIN ( + SELECT b.f1 + FROM tt3 b LEFT JOIN tt3 c ON (b.f1 = c.f1) + WHERE c.f1 IS NULL +) AS d ON (a.f1 = d.f1) +WHERE d.f1 IS NULL +-- !query analysis +Project [f1#x] ++- Filter isnull(f1#x) + +- Join LeftOuter, (f1#x = f1#x) + :- SubqueryAlias a + : +- SubqueryAlias tt4 + : +- View (`tt4`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias d + +- Project [f1#x] + +- Filter isnull(f1#x) + +- Join LeftOuter, (f1#x = f1#x) + :- SubqueryAlias b + : +- SubqueryAlias tt3 + : +- View (`tt3`, [f1#x,f2#x]) + : +- Project [cast(f1#x as int) AS f1#x, cast(f2#x as string) AS f2#x] + : +- Project [f1#x, f2#x] + : +- SubqueryAlias v + : +- Project [id#x AS f1#x, repeat(xyzzy, 100)#x AS f2#x] + : +- Project [cast(id#xL as int) AS id#x, repeat(xyzzy, 100) AS repeat(xyzzy, 100)#x] + : +- SubqueryAlias x + : +- Range (1, 10001, step=1, splits=None) + +- SubqueryAlias c + +- SubqueryAlias tt3 + +- View (`tt3`, [f1#x,f2#x]) + +- Project [cast(f1#x as int) AS f1#x, cast(f2#x as string) AS f2#x] + +- Project [f1#x, f2#x] + +- SubqueryAlias v + +- Project [id#x AS f1#x, repeat(xyzzy, 100)#x AS f2#x] + +- Project [cast(id#xL as int) AS id#x, repeat(xyzzy, 100) AS repeat(xyzzy, 100)#x] + +- SubqueryAlias x + +- Range (1, 10001, step=1, splits=None) + + +-- !query +create or replace temporary view tt5 as select * from + (values (1, 10), (1, 11)) + as v(f1, f2) +-- !query analysis +CreateViewCommand `tt5`, select * from + (values (1, 10), (1, 11)) + as v(f1, f2), false, true, LocalTempView, true + +- Project [f1#x, f2#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x, col2#x AS f2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +create or replace temporary view tt6 as select * from + (values (1, 9), (1, 2), (2, 9)) + as v(f1, f2) +-- !query analysis +CreateViewCommand `tt6`, select * from + (values (1, 9), (1, 2), (2, 9)) + as v(f1, f2), false, true, LocalTempView, true + +- Project [f1#x, f2#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x, col2#x AS f2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from tt5,tt6 where tt5.f1 = tt6.f1 and tt5.f1 = tt5.f2 - tt6.f2 +-- !query analysis +Project [f1#x, f2#x, f1#x, f2#x] ++- Filter ((f1#x = f1#x) AND (f1#x = (f2#x - f2#x))) + +- Join Inner + :- SubqueryAlias tt5 + : +- View (`tt5`, [f1#x,f2#x]) + : +- Project [cast(f1#x as int) AS f1#x, cast(f2#x as int) AS f2#x] + : +- Project [f1#x, f2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x, col2#x AS f2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias tt6 + +- View (`tt6`, [f1#x,f2#x]) + +- Project [cast(f1#x as int) AS f1#x, cast(f2#x as int) AS f2#x] + +- Project [f1#x, f2#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x, col2#x AS f2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +create or replace temporary view xx as select * from + (values (1), (2), (3)) + as v(pkxx) +-- !query analysis +CreateViewCommand `xx`, select * from + (values (1), (2), (3)) + as v(pkxx), false, true, LocalTempView, true + +- Project [pkxx#x] + +- SubqueryAlias v + +- Project [col1#x AS pkxx#x] + +- LocalRelation [col1#x] + + +-- !query +create or replace temporary view yy as select * from + (values (101, 1), (201, 2), (301, NULL)) + as v(pkyy, pkxx) +-- !query analysis +CreateViewCommand `yy`, select * from + (values (101, 1), (201, 2), (301, NULL)) + as v(pkyy, pkxx), false, true, LocalTempView, true + +- Project [pkyy#x, pkxx#x] + +- SubqueryAlias v + +- Project [col1#x AS pkyy#x, col2#x AS pkxx#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select yy.pkyy as yy_pkyy, yy.pkxx as yy_pkxx, yya.pkyy as yya_pkyy, + xxa.pkxx as xxa_pkxx, xxb.pkxx as xxb_pkxx +from yy + left join (SELECT * FROM yy where pkyy = 101) as yya ON yy.pkyy = yya.pkyy + left join xx xxa on yya.pkxx = xxa.pkxx + left join xx xxb on coalesce (xxa.pkxx, 1) = xxb.pkxx +-- !query analysis +Project [pkyy#x AS yy_pkyy#x, pkxx#x AS yy_pkxx#x, pkyy#x AS yya_pkyy#x, pkxx#x AS xxa_pkxx#x, pkxx#x AS xxb_pkxx#x] ++- Join LeftOuter, (coalesce(pkxx#x, 1) = pkxx#x) + :- Join LeftOuter, (pkxx#x = pkxx#x) + : :- Join LeftOuter, (pkyy#x = pkyy#x) + : : :- SubqueryAlias yy + : : : +- View (`yy`, [pkyy#x,pkxx#x]) + : : : +- Project [cast(pkyy#x as int) AS pkyy#x, cast(pkxx#x as int) AS pkxx#x] + : : : +- Project [pkyy#x, pkxx#x] + : : : +- SubqueryAlias v + : : : +- Project [col1#x AS pkyy#x, col2#x AS pkxx#x] + : : : +- LocalRelation [col1#x, col2#x] + : : +- SubqueryAlias yya + : : +- Project [pkyy#x, pkxx#x] + : : +- Filter (pkyy#x = 101) + : : +- SubqueryAlias yy + : : +- View (`yy`, [pkyy#x,pkxx#x]) + : : +- Project [cast(pkyy#x as int) AS pkyy#x, cast(pkxx#x as int) AS pkxx#x] + : : +- Project [pkyy#x, pkxx#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS pkyy#x, col2#x AS pkxx#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias xxa + : +- SubqueryAlias xx + : +- View (`xx`, [pkxx#x]) + : +- Project [cast(pkxx#x as int) AS pkxx#x] + : +- Project [pkxx#x] + : +- SubqueryAlias v + : +- Project [col1#x AS pkxx#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias xxb + +- SubqueryAlias xx + +- View (`xx`, [pkxx#x]) + +- Project [cast(pkxx#x as int) AS pkxx#x] + +- Project [pkxx#x] + +- SubqueryAlias v + +- Project [col1#x AS pkxx#x] + +- LocalRelation [col1#x] + + +-- !query +create or replace temporary view zt1 as select * from + (values (53)) + as v(f1) +-- !query analysis +CreateViewCommand `zt1`, select * from + (values (53)) + as v(f1), false, true, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +create or replace temporary view zt2 as select * from + (values (53)) + as v(f2) +-- !query analysis +CreateViewCommand `zt2`, select * from + (values (53)) + as v(f2), false, true, LocalTempView, true + +- Project [f2#x] + +- SubqueryAlias v + +- Project [col1#x AS f2#x] + +- LocalRelation [col1#x] + + +-- !query +create or replace temporary view zt3(f3 int) using parquet +-- !query analysis +CreateTempViewUsing [tableIdent:`zt3` StructType(StructField(f3,IntegerType,true)) replace:true provider:parquet Map() + + +-- !query +select * from + zt2 left join zt3 on (f2 = f3) + left join zt1 on (f3 = f1) +where f2 = 53 +-- !query analysis +Project [f2#x, f3#x, f1#x] ++- Filter (f2#x = 53) + +- Join LeftOuter, (f3#x = f1#x) + :- Join LeftOuter, (f2#x = f3#x) + : :- SubqueryAlias zt2 + : : +- View (`zt2`, [f2#x]) + : : +- Project [cast(f2#x as int) AS f2#x] + : : +- Project [f2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS f2#x] + : : +- LocalRelation [col1#x] + : +- SubqueryAlias zt3 + : +- View (`zt3`, [f3#x]) + : +- Relation [f3#x] parquet + +- SubqueryAlias zt1 + +- View (`zt1`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +create temp view zv1 as select *,'dummy' AS junk from zt1 +-- !query analysis +CreateViewCommand `zv1`, select *,'dummy' AS junk from zt1, false, false, LocalTempView, true + +- Project [f1#x, dummy AS junk#x] + +- SubqueryAlias zt1 + +- View (`zt1`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from + zt2 left join zt3 on (f2 = f3) + left join zv1 on (f3 = f1) +where f2 = 53 +-- !query analysis +Project [f2#x, f3#x, f1#x, junk#x] ++- Filter (f2#x = 53) + +- Join LeftOuter, (f3#x = f1#x) + :- Join LeftOuter, (f2#x = f3#x) + : :- SubqueryAlias zt2 + : : +- View (`zt2`, [f2#x]) + : : +- Project [cast(f2#x as int) AS f2#x] + : : +- Project [f2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS f2#x] + : : +- LocalRelation [col1#x] + : +- SubqueryAlias zt3 + : +- View (`zt3`, [f3#x]) + : +- Relation [f3#x] parquet + +- SubqueryAlias zv1 + +- View (`zv1`, [f1#x,junk#x]) + +- Project [cast(f1#x as int) AS f1#x, cast(junk#x as string) AS junk#x] + +- Project [f1#x, dummy AS junk#x] + +- SubqueryAlias zt1 + +- View (`zt1`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select a.unique2, a.ten, b.tenthous, b.unique2, b.hundred +from tenk1 a left join tenk1 b on a.unique2 = b.tenthous +where a.unique1 = 42 and + ((b.unique2 is null and a.ten = 2) or b.hundred = 3) +-- !query analysis +Project [unique2#x, ten#x, tenthous#x, unique2#x, hundred#x] ++- Filter ((unique1#x = 42) AND ((isnull(unique2#x) AND (ten#x = 2)) OR (hundred#x = 3))) + +- Join LeftOuter, (unique2#x = tenthous#x) + :- SubqueryAlias a + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +create or replace temporary view a (i integer) using parquet +-- !query analysis +CreateTempViewUsing [tableIdent:`a` StructType(StructField(i,IntegerType,true)) replace:true provider:parquet Map() + + +-- !query +create or replace temporary view b (x integer, y integer) using parquet +-- !query analysis +CreateTempViewUsing [tableIdent:`b` StructType(StructField(x,IntegerType,true),StructField(y,IntegerType,true)) replace:true provider:parquet Map() + + +-- !query +select * from a left join b on i = x and i = y and x = i +-- !query analysis +Project [i#x, x#x, y#x] ++- Join LeftOuter, (((i#x = x#x) AND (i#x = y#x)) AND (x#x = i#x)) + :- SubqueryAlias a + : +- View (`a`, [i#x]) + : +- Relation [i#x] parquet + +- SubqueryAlias b + +- View (`b`, [x#x,y#x]) + +- Relation [x#x,y#x] parquet + + +-- !query +select t1.q2, count(t2.q1, t2.q2) +from int8_tbl t1 left join int8_tbl t2 on (t1.q2 = t2.q1) +group by t1.q2 order by 1 +-- !query analysis +Sort [q2#xL ASC NULLS FIRST], true ++- Aggregate [q2#xL], [q2#xL, count(q1#xL, q2#xL) AS count(q1, q2)#xL] + +- Join LeftOuter, (q2#xL = q1#xL) + :- SubqueryAlias t1 + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias t2 + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select t1.q2, count(t2.q1, t2.q2) +from int8_tbl t1 left join (select * from int8_tbl) t2 on (t1.q2 = t2.q1) +group by t1.q2 order by 1 +-- !query analysis +Sort [q2#xL ASC NULLS FIRST], true ++- Aggregate [q2#xL], [q2#xL, count(q1#xL, q2#xL) AS count(q1, q2)#xL] + +- Join LeftOuter, (q2#xL = q1#xL) + :- SubqueryAlias t1 + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias t2 + +- Project [q1#xL, q2#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select t1.q2, count(t2.q1, t2.q2) +from int8_tbl t1 left join + (select q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2 + on (t1.q2 = t2.q1) +group by t1.q2 order by 1 +-- !query analysis +Sort [q2#xL ASC NULLS FIRST], true ++- Aggregate [q2#xL], [q2#xL, count(q1#xL, q2#xL) AS count(q1, q2)#xL] + +- Join LeftOuter, (q2#xL = q1#xL) + :- SubqueryAlias t1 + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias t2 + +- Project [q1#xL, CASE WHEN (q2#xL = cast(1 as bigint)) THEN cast(1 as bigint) ELSE q2#xL END AS q2#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +create or replace temporary view a as select * from + (values ('p'), ('q')) + as v(code) +-- !query analysis +CreateViewCommand `a`, select * from + (values ('p'), ('q')) + as v(code), false, true, LocalTempView, true + +- Project [code#x] + +- SubqueryAlias v + +- Project [col1#x AS code#x] + +- LocalRelation [col1#x] + + +-- !query +create or replace temporary view b as select * from + (values ('p', 1), ('p', 2)) + as v(a, num) +-- !query analysis +CreateViewCommand `b`, select * from + (values ('p', 1), ('p', 2)) + as v(a, num), false, true, LocalTempView, true + +- Project [a#x, num#x] + +- SubqueryAlias v + +- Project [col1#x AS a#x, col2#x AS num#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +create or replace temporary view c as select * from + (values ('A', 'p'), ('B', 'q'), ('C', null)) + as v(name, a) +-- !query analysis +CreateViewCommand `c`, select * from + (values ('A', 'p'), ('B', 'q'), ('C', null)) + as v(name, a), false, true, LocalTempView, true + +- Project [name#x, a#x] + +- SubqueryAlias v + +- Project [col1#x AS name#x, col2#x AS a#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select c.name, ss.code, ss.b_cnt, ss.const +from c left join + (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const + from a left join + (select count(1) as cnt, b.a from b group by b.a) as b_grp + on a.code = b_grp.a + ) as ss + on (c.a = ss.code) +order by c.name +-- !query analysis +Sort [name#x ASC NULLS FIRST], true ++- Project [name#x, code#x, b_cnt#xL, const#x] + +- Join LeftOuter, (a#x = code#x) + :- SubqueryAlias c + : +- View (`c`, [name#x,a#x]) + : +- Project [cast(name#x as string) AS name#x, cast(a#x as string) AS a#x] + : +- Project [name#x, a#x] + : +- SubqueryAlias v + : +- Project [col1#x AS name#x, col2#x AS a#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias ss + +- Project [code#x, coalesce(cnt#xL, cast(0 as bigint)) AS b_cnt#xL, -1 AS const#x] + +- Join LeftOuter, (code#x = a#x) + :- SubqueryAlias a + : +- View (`a`, [code#x]) + : +- Project [cast(code#x as string) AS code#x] + : +- Project [code#x] + : +- SubqueryAlias v + : +- Project [col1#x AS code#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias b_grp + +- Aggregate [a#x], [count(1) AS cnt#xL, a#x] + +- SubqueryAlias b + +- View (`b`, [a#x,num#x]) + +- Project [cast(a#x as string) AS a#x, cast(num#x as int) AS num#x] + +- Project [a#x, num#x] + +- SubqueryAlias v + +- Project [col1#x AS a#x, col2#x AS num#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT sub3.key3, sub4.value2, COALESCE(sub4.value2, 66) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON sub5.key5 = sub6.key6 + ) sub4 + ON sub4.key5 = sub3.key3 +) sub2 +ON sub1.key1 = sub2.key3 +-- !query analysis +Project [key1#x, key3#x, value2#x, value3#x] ++- Join LeftOuter, (key1#x = key3#x) + :- SubqueryAlias sub1 + : +- Project [1 AS key1#x] + : +- OneRowRelation + +- SubqueryAlias sub2 + +- Project [key3#x, value2#x, coalesce(value2#x, 66) AS value3#x] + +- Join LeftOuter, (key5#x = key3#x) + :- SubqueryAlias sub3 + : +- Project [1 AS key3#x] + : +- OneRowRelation + +- SubqueryAlias sub4 + +- Project [key5#x, coalesce(value1#x, 1) AS value2#x] + +- Join LeftOuter, (key5#x = key6#x) + :- SubqueryAlias sub5 + : +- Project [1 AS key5#x] + : +- OneRowRelation + +- SubqueryAlias sub6 + +- Project [2 AS key6#x, 42 AS value1#x] + +- OneRowRelation + + +-- !query +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT sub3.key3, value2, COALESCE(value2, 66) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON sub5.key5 = sub6.key6 + ) sub4 + ON sub4.key5 = sub3.key3 +) sub2 +ON sub1.key1 = sub2.key3 +-- !query analysis +Project [key1#x, key3#x, value2#x, value3#x] ++- Join LeftOuter, (key1#x = key3#x) + :- SubqueryAlias sub1 + : +- Project [1 AS key1#x] + : +- OneRowRelation + +- SubqueryAlias sub2 + +- Project [key3#x, value2#x, coalesce(value2#x, 66) AS value3#x] + +- Join LeftOuter, (key5#x = key3#x) + :- SubqueryAlias sub3 + : +- Project [1 AS key3#x] + : +- OneRowRelation + +- SubqueryAlias sub4 + +- Project [key5#x, coalesce(value1#x, 1) AS value2#x] + +- Join LeftOuter, (key5#x = key6#x) + :- SubqueryAlias sub5 + : +- Project [1 AS key5#x] + : +- OneRowRelation + +- SubqueryAlias sub6 + +- Project [2 AS key6#x, 42 AS value1#x] + +- OneRowRelation + + +-- !query +SELECT qq, unique1 + FROM + ( SELECT COALESCE(q1, 0) AS qq FROM int8_tbl a ) AS ss1 + FULL OUTER JOIN + ( SELECT COALESCE(q2, -1) AS qq FROM int8_tbl b ) AS ss2 + USING (qq) + INNER JOIN tenk1 c ON qq = unique2 +-- !query analysis +Project [qq#xL, unique1#x] ++- Join Inner, (qq#xL = cast(unique2#x as bigint)) + :- Project [coalesce(qq#xL, qq#xL) AS qq#xL] + : +- Join FullOuter, (qq#xL = qq#xL) + : :- SubqueryAlias ss1 + : : +- Project [coalesce(q1#xL, cast(0 as bigint)) AS qq#xL] + : : +- SubqueryAlias a + : : +- SubqueryAlias int8_tbl + : : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : : +- Project [q1#xL, q2#xL] + : : +- SubqueryAlias v + : : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : : +- LocalRelation [col1#xL, col2#xL] + : +- SubqueryAlias ss2 + : +- Project [coalesce(q2#xL, cast(-1 as bigint)) AS qq#xL] + : +- SubqueryAlias b + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias c + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +create or replace temporary view nt1 as select * from + (values(1,true,true), (2,true,false), (3,false,false)) + as v(id, a1, a2) +-- !query analysis +CreateViewCommand `nt1`, select * from + (values(1,true,true), (2,true,false), (3,false,false)) + as v(id, a1, a2), false, true, LocalTempView, true + +- Project [id#x, a1#x, a2#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x, col2#x AS a1#x, col3#x AS a2#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +create or replace temporary view nt2 as select * from + (values(1,1,true,true), (2,2,true,false), (3,3,false,false)) + as v(id, nt1_id, b1, b2) +-- !query analysis +CreateViewCommand `nt2`, select * from + (values(1,1,true,true), (2,2,true,false), (3,3,false,false)) + as v(id, nt1_id, b1, b2), false, true, LocalTempView, true + +- Project [id#x, nt1_id#x, b1#x, b2#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x, col2#x AS nt1_id#x, col3#x AS b1#x, col4#x AS b2#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +create or replace temporary view nt3 as select * from + (values(1,1,true), (2,2,false), (3,3,true)) + as v(id, nt2_id, c1) +-- !query analysis +CreateViewCommand `nt3`, select * from + (values(1,1,true), (2,2,false), (3,3,true)) + as v(id, nt2_id, c1), false, true, LocalTempView, true + +- Project [id#x, nt2_id#x, c1#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x, col2#x AS nt2_id#x, col3#x AS c1#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +select nt3.id +from nt3 as nt3 + left join + (select nt2.*, (nt2.b1 and ss1.a3) AS b3 + from nt2 as nt2 + left join + (select nt1.*, (nt1.id is not null) as a3 from nt1) as ss1 + on ss1.id = nt2.nt1_id + ) as ss2 + on ss2.id = nt3.nt2_id +where nt3.id = 1 and ss2.b3 +-- !query analysis +Project [id#x] ++- Filter ((id#x = 1) AND b3#x) + +- Join LeftOuter, (id#x = nt2_id#x) + :- SubqueryAlias nt3 + : +- SubqueryAlias nt3 + : +- View (`nt3`, [id#x,nt2_id#x,c1#x]) + : +- Project [cast(id#x as int) AS id#x, cast(nt2_id#x as int) AS nt2_id#x, cast(c1#x as boolean) AS c1#x] + : +- Project [id#x, nt2_id#x, c1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS id#x, col2#x AS nt2_id#x, col3#x AS c1#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias ss2 + +- Project [id#x, nt1_id#x, b1#x, b2#x, (b1#x AND a3#x) AS b3#x] + +- Join LeftOuter, (id#x = nt1_id#x) + :- SubqueryAlias nt2 + : +- SubqueryAlias nt2 + : +- View (`nt2`, [id#x,nt1_id#x,b1#x,b2#x]) + : +- Project [cast(id#x as int) AS id#x, cast(nt1_id#x as int) AS nt1_id#x, cast(b1#x as boolean) AS b1#x, cast(b2#x as boolean) AS b2#x] + : +- Project [id#x, nt1_id#x, b1#x, b2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS id#x, col2#x AS nt1_id#x, col3#x AS b1#x, col4#x AS b2#x] + : +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + +- SubqueryAlias ss1 + +- Project [id#x, a1#x, a2#x, isnotnull(id#x) AS a3#x] + +- SubqueryAlias nt1 + +- View (`nt1`, [id#x,a1#x,a2#x]) + +- Project [cast(id#x as int) AS id#x, cast(a1#x as boolean) AS a1#x, cast(a2#x as boolean) AS a2#x] + +- Project [id#x, a1#x, a2#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x, col2#x AS a1#x, col3#x AS a2#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +select * from int4_tbl a full join int4_tbl b on true +-- !query analysis +Project [f1#x, f1#x] ++- Join FullOuter, true + :- SubqueryAlias a + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias b + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from int4_tbl a full join int4_tbl b on false +-- !query analysis +Project [f1#x, f1#x] ++- Join FullOuter, false + :- SubqueryAlias a + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias b + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select count(*) from + tenk1 a join tenk1 b on a.unique1 = b.unique2 + left join tenk1 c on a.unique2 = b.unique1 and c.thousand = a.thousand + join int4_tbl on b.thousand = f1 +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Join Inner, (thousand#x = f1#x) + :- Join LeftOuter, ((unique2#x = unique1#x) AND (thousand#x = thousand#x)) + : :- Join Inner, (unique1#x = unique2#x) + : : :- SubqueryAlias a + : : : +- SubqueryAlias spark_catalog.default.tenk1 + : : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : : +- SubqueryAlias b + : : +- SubqueryAlias spark_catalog.default.tenk1 + : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : +- SubqueryAlias c + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select b.unique1 from + tenk1 a join tenk1 b on a.unique1 = b.unique2 + left join tenk1 c on b.unique1 = 42 and c.thousand = a.thousand + join int4_tbl i1 on b.thousand = f1 + right join int4_tbl i2 on i2.f1 = b.tenthous + order by 1 +-- !query analysis +Sort [unique1#x ASC NULLS FIRST], true ++- Project [unique1#x] + +- Join RightOuter, (f1#x = tenthous#x) + :- Join Inner, (thousand#x = f1#x) + : :- Join LeftOuter, ((unique1#x = 42) AND (thousand#x = thousand#x)) + : : :- Join Inner, (unique1#x = unique2#x) + : : : :- SubqueryAlias a + : : : : +- SubqueryAlias spark_catalog.default.tenk1 + : : : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : : : +- SubqueryAlias b + : : : +- SubqueryAlias spark_catalog.default.tenk1 + : : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : : +- SubqueryAlias c + : : +- SubqueryAlias spark_catalog.default.tenk1 + : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : +- SubqueryAlias i1 + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias i2 + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from +( + select unique1, q1, coalesce(unique1, -1) + q1 as fault + from int8_tbl left join tenk1 on (q2 = unique2) +) ss +where fault = 122 +order by fault +-- !query analysis +Sort [fault#xL ASC NULLS FIRST], true ++- Project [unique1#x, q1#xL, fault#xL] + +- Filter (fault#xL = cast(122 as bigint)) + +- SubqueryAlias ss + +- Project [unique1#x, q1#xL, (cast(coalesce(unique1#x, -1) as bigint) + q1#xL) AS fault#xL] + +- Join LeftOuter, (q2#xL = cast(unique2#x as bigint)) + :- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select q1, unique2, thousand, hundred + from int8_tbl a left join tenk1 b on q1 = unique2 + where coalesce(thousand,123) = q1 and q1 = coalesce(hundred,123) +-- !query analysis +Project [q1#xL, unique2#x, thousand#x, hundred#x] ++- Filter ((cast(coalesce(thousand#x, 123) as bigint) = q1#xL) AND (q1#xL = cast(coalesce(hundred#x, 123) as bigint))) + +- Join LeftOuter, (q1#xL = cast(unique2#x as bigint)) + :- SubqueryAlias a + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select f1, unique2, case when unique2 is null then f1 else 0 end + from int4_tbl a left join tenk1 b on f1 = unique2 + where (case when unique2 is null then f1 else 0 end) = 0 +-- !query analysis +Project [f1#x, unique2#x, CASE WHEN isnull(unique2#x) THEN f1#x ELSE 0 END AS CASE WHEN (unique2 IS NULL) THEN f1 ELSE 0 END#x] ++- Filter (CASE WHEN isnull(unique2#x) THEN f1#x ELSE 0 END = 0) + +- Join LeftOuter, (f1#x = unique2#x) + :- SubqueryAlias a + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select a.unique1, b.unique1, c.unique1, coalesce(b.twothousand, a.twothousand) + from tenk1 a left join tenk1 b on b.thousand = a.unique1 left join tenk1 c on c.unique2 = coalesce(b.twothousand, a.twothousand) + where a.unique2 < 10 and coalesce(b.twothousand, a.twothousand) = 44 +-- !query analysis +Project [unique1#x, unique1#x, unique1#x, coalesce(twothousand#x, twothousand#x) AS coalesce(twothousand, twothousand)#x] ++- Filter ((unique2#x < 10) AND (coalesce(twothousand#x, twothousand#x) = 44)) + +- Join LeftOuter, (unique2#x = coalesce(twothousand#x, twothousand#x)) + :- Join LeftOuter, (thousand#x = unique1#x) + : :- SubqueryAlias a + : : +- SubqueryAlias spark_catalog.default.tenk1 + : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : +- SubqueryAlias b + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias c + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select * from + text_tbl t1 + inner join int8_tbl i8 + on i8.q2 = 456 + right join text_tbl t2 + on t1.f1 = 'doh!' + left join int4_tbl i4 + on i8.q1 = i4.f1 +-- !query analysis +Project [f1#x, q1#xL, q2#xL, f1#x, f1#x] ++- Join LeftOuter, (q1#xL = cast(f1#x as bigint)) + :- Join RightOuter, (f1#x = doh!) + : :- Join Inner, (q2#xL = cast(456 as bigint)) + : : :- SubqueryAlias t1 + : : : +- SubqueryAlias text_tbl + : : : +- View (`TEXT_TBL`, [f1#x]) + : : : +- Project [cast(f1#x as string) AS f1#x] + : : : +- Project [f1#x] + : : : +- SubqueryAlias v + : : : +- Project [col1#x AS f1#x] + : : : +- LocalRelation [col1#x] + : : +- SubqueryAlias i8 + : : +- SubqueryAlias int8_tbl + : : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : : +- Project [q1#xL, q2#xL] + : : +- SubqueryAlias v + : : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : : +- LocalRelation [col1#xL, col2#xL] + : +- SubqueryAlias t2 + : +- SubqueryAlias text_tbl + : +- View (`TEXT_TBL`, [f1#x]) + : +- Project [cast(f1#x as string) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias i4 + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from + (select 1 as id) as xx + left join + (tenk1 as a1 full join (select 1 as id) as yy on (a1.unique1 = yy.id)) + on (xx.id = coalesce(yy.id)) +-- !query analysis +Project [id#x, unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, id#x] ++- Join LeftOuter, (id#x = coalesce(id#x)) + :- SubqueryAlias xx + : +- Project [1 AS id#x] + : +- OneRowRelation + +- Join FullOuter, (unique1#x = id#x) + :- SubqueryAlias a1 + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias yy + +- Project [1 AS id#x] + +- OneRowRelation + + +-- !query +select a.q2, b.q1 + from int8_tbl a left join int8_tbl b on a.q2 = coalesce(b.q1, 1) + where coalesce(b.q1, 1) > 0 +-- !query analysis +Project [q2#xL, q1#xL] ++- Filter (coalesce(q1#xL, cast(1 as bigint)) > cast(0 as bigint)) + +- Join LeftOuter, (q2#xL = coalesce(q1#xL, cast(1 as bigint))) + :- SubqueryAlias a + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias b + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +create or replace temporary view parent as select * from + (values (1, 10), (2, 20), (3, 30)) + as v(k, pd) +-- !query analysis +CreateViewCommand `parent`, select * from + (values (1, 10), (2, 20), (3, 30)) + as v(k, pd), false, true, LocalTempView, true + +- Project [k#x, pd#x] + +- SubqueryAlias v + +- Project [col1#x AS k#x, col2#x AS pd#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +create or replace temporary view child as select * from + (values (1, 100), (4, 400)) + as v(k, cd) +-- !query analysis +CreateViewCommand `child`, select * from + (values (1, 100), (4, 400)) + as v(k, cd), false, true, LocalTempView, true + +- Project [k#x, cd#x] + +- SubqueryAlias v + +- Project [col1#x AS k#x, col2#x AS cd#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select p.* from parent p left join child c on (p.k = c.k) +-- !query analysis +Project [k#x, pd#x] ++- Join LeftOuter, (k#x = k#x) + :- SubqueryAlias p + : +- SubqueryAlias parent + : +- View (`parent`, [k#x,pd#x]) + : +- Project [cast(k#x as int) AS k#x, cast(pd#x as int) AS pd#x] + : +- Project [k#x, pd#x] + : +- SubqueryAlias v + : +- Project [col1#x AS k#x, col2#x AS pd#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias c + +- SubqueryAlias child + +- View (`child`, [k#x,cd#x]) + +- Project [cast(k#x as int) AS k#x, cast(cd#x as int) AS cd#x] + +- Project [k#x, cd#x] + +- SubqueryAlias v + +- Project [col1#x AS k#x, col2#x AS cd#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select p.*, linked from parent p + left join (select c.*, true as linked from child c) as ss + on (p.k = ss.k) +-- !query analysis +Project [k#x, pd#x, linked#x] ++- Join LeftOuter, (k#x = k#x) + :- SubqueryAlias p + : +- SubqueryAlias parent + : +- View (`parent`, [k#x,pd#x]) + : +- Project [cast(k#x as int) AS k#x, cast(pd#x as int) AS pd#x] + : +- Project [k#x, pd#x] + : +- SubqueryAlias v + : +- Project [col1#x AS k#x, col2#x AS pd#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias ss + +- Project [k#x, cd#x, true AS linked#x] + +- SubqueryAlias c + +- SubqueryAlias child + +- View (`child`, [k#x,cd#x]) + +- Project [cast(k#x as int) AS k#x, cast(cd#x as int) AS cd#x] + +- Project [k#x, cd#x] + +- SubqueryAlias v + +- Project [col1#x AS k#x, col2#x AS cd#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select p.* from + parent p left join child c on (p.k = c.k) + where p.k = 1 and p.k = 2 +-- !query analysis +Project [k#x, pd#x] ++- Filter ((k#x = 1) AND (k#x = 2)) + +- Join LeftOuter, (k#x = k#x) + :- SubqueryAlias p + : +- SubqueryAlias parent + : +- View (`parent`, [k#x,pd#x]) + : +- Project [cast(k#x as int) AS k#x, cast(pd#x as int) AS pd#x] + : +- Project [k#x, pd#x] + : +- SubqueryAlias v + : +- Project [col1#x AS k#x, col2#x AS pd#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias c + +- SubqueryAlias child + +- View (`child`, [k#x,cd#x]) + +- Project [cast(k#x as int) AS k#x, cast(cd#x as int) AS cd#x] + +- Project [k#x, cd#x] + +- SubqueryAlias v + +- Project [col1#x AS k#x, col2#x AS cd#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select p.* from + (parent p left join child c on (p.k = c.k)) join parent x on p.k = x.k + where p.k = 1 and p.k = 2 +-- !query analysis +Project [k#x, pd#x] ++- Filter ((k#x = 1) AND (k#x = 2)) + +- Join Inner, (k#x = k#x) + :- Join LeftOuter, (k#x = k#x) + : :- SubqueryAlias p + : : +- SubqueryAlias parent + : : +- View (`parent`, [k#x,pd#x]) + : : +- Project [cast(k#x as int) AS k#x, cast(pd#x as int) AS pd#x] + : : +- Project [k#x, pd#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS k#x, col2#x AS pd#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias c + : +- SubqueryAlias child + : +- View (`child`, [k#x,cd#x]) + : +- Project [cast(k#x as int) AS k#x, cast(cd#x as int) AS cd#x] + : +- Project [k#x, cd#x] + : +- SubqueryAlias v + : +- Project [col1#x AS k#x, col2#x AS cd#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias x + +- SubqueryAlias parent + +- View (`parent`, [k#x,pd#x]) + +- Project [cast(k#x as int) AS k#x, cast(pd#x as int) AS pd#x] + +- Project [k#x, pd#x] + +- SubqueryAlias v + +- Project [col1#x AS k#x, col2#x AS pd#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +create or replace temporary view a as select * from + (values (0), (1)) + as v(id) +-- !query analysis +CreateViewCommand `a`, select * from + (values (0), (1)) + as v(id), false, true, LocalTempView, true + +- Project [id#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x] + +- LocalRelation [col1#x] + + +-- !query +create or replace temporary view b as select * from + (values (0, 0), (1, NULL)) + as v(id, a_id) +-- !query analysis +CreateViewCommand `b`, select * from + (values (0, 0), (1, NULL)) + as v(id, a_id), false, true, LocalTempView, true + +- Project [id#x, a_id#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x, col2#x AS a_id#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0) +-- !query analysis +Project [id#x, a_id#x, id#x] ++- Filter (isnull(id#x) OR (id#x > 0)) + +- Join LeftOuter, (a_id#x = id#x) + :- SubqueryAlias b + : +- View (`b`, [id#x,a_id#x]) + : +- Project [cast(id#x as int) AS id#x, cast(a_id#x as int) AS a_id#x] + : +- Project [id#x, a_id#x] + : +- SubqueryAlias v + : +- Project [col1#x AS id#x, col2#x AS a_id#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias a + +- View (`a`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT b.* FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0) +-- !query analysis +Project [id#x, a_id#x] ++- Filter (isnull(id#x) OR (id#x > 0)) + +- Join LeftOuter, (a_id#x = id#x) + :- SubqueryAlias b + : +- View (`b`, [id#x,a_id#x]) + : +- Project [cast(id#x as int) AS id#x, cast(a_id#x as int) AS a_id#x] + : +- Project [id#x, a_id#x] + : +- SubqueryAlias v + : +- Project [col1#x AS id#x, col2#x AS a_id#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias a + +- View (`a`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x] + +- LocalRelation [col1#x] + + +-- !query +create or replace temporary view innertab as select * from + (values (123L, 42L)) + as v(id, dat1) +-- !query analysis +CreateViewCommand `innertab`, select * from + (values (123L, 42L)) + as v(id, dat1), false, true, LocalTempView, true + +- Project [id#xL, dat1#xL] + +- SubqueryAlias v + +- Project [col1#xL AS id#xL, col2#xL AS dat1#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +SELECT * FROM + (SELECT 1 AS x) ss1 + LEFT JOIN + (SELECT q1, q2, COALESCE(dat1, q1) AS y + FROM int8_tbl LEFT JOIN innertab ON q2 = id) ss2 + ON true +-- !query analysis +Project [x#x, q1#xL, q2#xL, y#xL] ++- Join LeftOuter, true + :- SubqueryAlias ss1 + : +- Project [1 AS x#x] + : +- OneRowRelation + +- SubqueryAlias ss2 + +- Project [q1#xL, q2#xL, coalesce(dat1#xL, q1#xL) AS y#xL] + +- Join LeftOuter, (q2#xL = id#xL) + :- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias innertab + +- View (`innertab`, [id#xL,dat1#xL]) + +- Project [cast(id#xL as bigint) AS id#xL, cast(dat1#xL as bigint) AS dat1#xL] + +- Project [id#xL, dat1#xL] + +- SubqueryAlias v + +- Project [col1#xL AS id#xL, col2#xL AS dat1#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = f1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`f1`", + "referenceNames" : "[`j`.`f1`, `j`.`f1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 78, + "stopIndex" : 79, + "fragment" : "f1" + } ] +} + + +-- !query +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = y.f1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`.`f1`", + "proposal" : "`j`.`f1`, `j`.`f1`, `x`.`q1`, `x`.`q2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 78, + "stopIndex" : 81, + "fragment" : "y.f1" + } ] +} + + +-- !query +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on q1 = f1 +-- !query analysis +Project [q1#xL, q2#xL, f1#x, ff#x] ++- Join Inner, (q1#xL = cast(f1#x as bigint)) + :- SubqueryAlias x + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias j + +- Join Cross + :- SubqueryAlias x + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias y + +- Project [f1#x AS ff#x] + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select t1.uunique1 from + tenk1 t1 join tenk2 t2 on t1.two = t2.two +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1`.`uunique1`", + "proposal" : "`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`hundred`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "t1.uunique1" + } ] +} + + +-- !query +select t2.uunique1 from + tenk1 t1 join tenk2 t2 on t1.two = t2.two +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t2`.`uunique1`", + "proposal" : "`t2`.`unique1`, `t1`.`unique1`, `t2`.`unique2`, `t1`.`unique2`, `t2`.`hundred`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "t2.uunique1" + } ] +} + + +-- !query +select uunique1 from + tenk1 t1 join tenk2 t2 on t1.two = t2.two +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`uunique1`", + "proposal" : "`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`even`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "uunique1" + } ] +} + + +-- !query +select unique2, x.* +from tenk1 a, lateral (select * from int4_tbl b where f1 = a.unique1) x +-- !query analysis +Project [unique2#x, f1#x] ++- LateralJoin lateral-subquery#x [unique1#x], Inner + : +- SubqueryAlias x + : +- Project [f1#x] + : +- Filter (f1#x = outer(unique1#x)) + : +- SubqueryAlias b + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias a + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select unique2, x.* +from int4_tbl x, lateral (select unique2 from tenk1 where f1 = unique1) ss +-- !query analysis +Project [unique2#x, f1#x] ++- LateralJoin lateral-subquery#x [f1#x], Inner + : +- SubqueryAlias ss + : +- Project [unique2#x] + : +- Filter (outer(f1#x) = unique1#x) + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias x + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select unique2, x.* +from int4_tbl x left join lateral (select unique1, unique2 from tenk1 where f1 = unique1) ss on true +-- !query analysis +Project [unique2#x, f1#x] ++- LateralJoin lateral-subquery#x [f1#x], LeftOuter, true + : +- SubqueryAlias ss + : +- Project [unique1#x, unique2#x] + : +- Filter (outer(f1#x) = unique1#x) + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias x + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from (select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1, + lateral (select x) ss2(y) +-- !query analysis +Project [x#x, f1#x, y#x] ++- LateralJoin lateral-subquery#x [x#x], Inner + : +- SubqueryAlias ss2 + : +- Project [x#x AS y#x] + : +- Project [outer(x#x)] + : +- OneRowRelation + +- Join Inner, (x#x = cast(f1#x as double)) + :- SubqueryAlias ss1 + : +- Project [(cast(f1#x as double) / cast(2 as double)) AS x#x] + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias i4 + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from ((select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1) j, + lateral (select x) ss2(y) +-- !query analysis +Project [x#x, f1#x, y#x] ++- LateralJoin lateral-subquery#x [x#x], Inner + : +- SubqueryAlias ss2 + : +- Project [x#x AS y#x] + : +- Project [outer(x#x)] + : +- OneRowRelation + +- SubqueryAlias j + +- Join Inner, (x#x = cast(f1#x as double)) + :- SubqueryAlias ss1 + : +- Project [(cast(f1#x as double) / cast(2 as double)) AS x#x] + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias i4 + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from + int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, + lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2) +-- !query analysis +Project [q1#xL, q2#xL, q1#xL, q2#xL, xq1#xL, yq1#xL, yq2#xL] ++- LateralJoin lateral-subquery#x [q1#xL && q1#xL && q2#xL], Inner + : +- SubqueryAlias v + : +- Project [q1#xL AS xq1#xL, q1#xL AS yq1#xL, q2#xL AS yq2#xL] + : +- Project [outer(q1#xL), outer(q1#xL), outer(q2#xL)] + : +- OneRowRelation + +- Join LeftOuter, (q2#xL = q1#xL) + :- SubqueryAlias x + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias y + +- Project [q1#xL, coalesce(q2#xL, cast(0 as bigint)) AS q2#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select x.* from + int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, + lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2) +-- !query analysis +Project [q1#xL, q2#xL] ++- LateralJoin lateral-subquery#x [q1#xL && q1#xL && q2#xL], Inner + : +- SubqueryAlias v + : +- Project [q1#xL AS xq1#xL, q1#xL AS yq1#xL, q2#xL AS yq2#xL] + : +- Project [outer(q1#xL), outer(q1#xL), outer(q2#xL)] + : +- OneRowRelation + +- Join LeftOuter, (q2#xL = q1#xL) + :- SubqueryAlias x + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias y + +- Project [q1#xL, coalesce(q2#xL, cast(0 as bigint)) AS q2#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select * from + int8_tbl a left join + lateral (select *, a.q2 as x from int8_tbl b) ss on a.q2 = ss.q1 +-- !query analysis +Project [q1#xL, q2#xL, q1#xL, q2#xL, x#xL] ++- LateralJoin lateral-subquery#x [q2#xL], LeftOuter, (q2#xL = q1#xL) + : +- SubqueryAlias ss + : +- Project [q1#xL, q2#xL, outer(q2#xL) AS x#xL] + : +- SubqueryAlias b + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias a + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select * from + int8_tbl a left join + lateral (select *, coalesce(a.q2, 42) as x from int8_tbl b) ss on a.q2 = ss.q1 +-- !query analysis +Project [q1#xL, q2#xL, q1#xL, q2#xL, x#xL] ++- LateralJoin lateral-subquery#x [q2#xL], LeftOuter, (q2#xL = q1#xL) + : +- SubqueryAlias ss + : +- Project [q1#xL, q2#xL, coalesce(outer(q2#xL), cast(42 as bigint)) AS x#xL] + : +- SubqueryAlias b + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias a + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select * from int4_tbl i left join + lateral (select * from int2_tbl j where i.f1 = j.f1) k on true +-- !query analysis +Project [f1#x, f1#x] ++- LateralJoin lateral-subquery#x [f1#x], LeftOuter, true + : +- SubqueryAlias k + : +- Project [f1#x] + : +- Filter (outer(f1#x) = cast(f1#x as int)) + : +- SubqueryAlias j + : +- SubqueryAlias int2_tbl + : +- View (`INT2_TBL`, [f1#x]) + : +- Project [cast(col1#x as smallint) AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias i + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from + int8_tbl a left join lateral + (select b.q1 as bq1, c.q1 as cq1, least(a.q1,b.q1,c.q1) from + int8_tbl b cross join int8_tbl c) ss + on a.q2 = ss.bq1 +-- !query analysis +Project [q1#xL, q2#xL, bq1#xL, cq1#xL, least(outer(a.q1), q1, q1)#xL] ++- LateralJoin lateral-subquery#x [q1#xL], LeftOuter, (q2#xL = bq1#xL) + : +- SubqueryAlias ss + : +- Project [q1#xL AS bq1#xL, q1#xL AS cq1#xL, least(outer(q1#xL), q1#xL, q1#xL) AS least(outer(a.q1), q1, q1)#xL] + : +- Join Cross + : :- SubqueryAlias b + : : +- SubqueryAlias int8_tbl + : : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : : +- Project [q1#xL, q2#xL] + : : +- SubqueryAlias v + : : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : : +- LocalRelation [col1#xL, col2#xL] + : +- SubqueryAlias c + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias a + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select f1,g from int4_tbl a, (select f1 as g) ss +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 38, + "stopIndex" : 39, + "fragment" : "f1" + } ] +} + + +-- !query +select f1,g from int4_tbl a, (select a.f1 as g) ss +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`.`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 38, + "stopIndex" : 41, + "fragment" : "a.f1" + } ] +} + + +-- !query +select f1,g from int4_tbl a cross join (select f1 as g) ss +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 48, + "stopIndex" : 49, + "fragment" : "f1" + } ] +} + + +-- !query +select f1,g from int4_tbl a cross join (select a.f1 as g) ss +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`.`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 48, + "stopIndex" : 51, + "fragment" : "a.f1" + } ] +} + + +-- !query +CREATE TABLE j1 (id1 int, id2 int) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`j1`, false + + +-- !query +CREATE TABLE j2 (id1 int, id2 int) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`j2`, false + + +-- !query +INSERT INTO j1 values(1,1),(1,2) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1], Append, `spark_catalog`.`default`.`j1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1), [id1, id2] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO j2 values(1,1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2], Append, `spark_catalog`.`default`.`j2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2), [id1, id2] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO j2 values(1,2) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2], Append, `spark_catalog`.`default`.`j2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2), [id1, id2] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from j1 +inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2 +where j1.id1 % 1000 = 1 and j2.id1 % 1000 = 1 +-- !query analysis +Project [id1#x, id2#x, id1#x, id2#x] ++- Filter (((id1#x % 1000) = 1) AND ((id1#x % 1000) = 1)) + +- Join Inner, ((id1#x = id1#x) AND (id2#x = id2#x)) + :- SubqueryAlias spark_catalog.default.j1 + : +- Relation spark_catalog.default.j1[id1#x,id2#x] parquet + +- SubqueryAlias spark_catalog.default.j2 + +- Relation spark_catalog.default.j2[id1#x,id2#x] parquet + + +-- !query +drop table j1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`j1`, false, false, false + + +-- !query +drop table j2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`j2`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/limit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/limit.sql.out new file mode 100644 index 0000000000000..8964e7f2340d0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/limit.sql.out @@ -0,0 +1,189 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT '' AS two, unique1, unique2, stringu1 + FROM onek WHERE unique1 > 50 + ORDER BY unique1 LIMIT 2 +-- !query analysis +GlobalLimit 2 ++- LocalLimit 2 + +- Sort [unique1#x ASC NULLS FIRST], true + +- Project [ AS two#x, unique1#x, unique2#x, stringu1#x] + +- Filter (unique1#x > 50) + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT '' AS five, unique1, unique2, stringu1 + FROM onek WHERE unique1 > 60 + ORDER BY unique1 LIMIT 5 +-- !query analysis +GlobalLimit 5 ++- LocalLimit 5 + +- Sort [unique1#x ASC NULLS FIRST], true + +- Project [ AS five#x, unique1#x, unique2#x, stringu1#x] + +- Filter (unique1#x > 60) + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT '' AS two, unique1, unique2, stringu1 + FROM onek WHERE unique1 > 60 AND unique1 < 63 + ORDER BY unique1 LIMIT 5 +-- !query analysis +GlobalLimit 5 ++- LocalLimit 5 + +- Sort [unique1#x ASC NULLS FIRST], true + +- Project [ AS two#x, unique1#x, unique2#x, stringu1#x] + +- Filter ((unique1#x > 60) AND (unique1#x < 63)) + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT '' AS three, unique1, unique2, stringu1 + FROM onek WHERE unique1 > 100 + ORDER BY unique1 LIMIT 3 OFFSET 20 +-- !query analysis +GlobalLimit 3 ++- LocalLimit 3 + +- Offset 20 + +- Sort [unique1#x ASC NULLS FIRST], true + +- Project [ AS three#x, unique1#x, unique2#x, stringu1#x] + +- Filter (unique1#x > 100) + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT '' AS zero, unique1, unique2, stringu1 + FROM onek WHERE unique1 < 50 + ORDER BY unique1 DESC LIMIT 8 OFFSET 99 +-- !query analysis +GlobalLimit 8 ++- LocalLimit 8 + +- Offset 99 + +- Sort [unique1#x DESC NULLS LAST], true + +- Project [ AS zero#x, unique1#x, unique2#x, stringu1#x] + +- Filter (unique1#x < 50) + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT '' AS eleven, unique1, unique2, stringu1 + FROM onek WHERE unique1 < 50 + ORDER BY unique1 DESC LIMIT 20 OFFSET 39 +-- !query analysis +GlobalLimit 20 ++- LocalLimit 20 + +- Offset 39 + +- Sort [unique1#x DESC NULLS LAST], true + +- Project [ AS eleven#x, unique1#x, unique2#x, stringu1#x] + +- Filter (unique1#x < 50) + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT '' AS ten, unique1, unique2, stringu1 + FROM onek + ORDER BY unique1 OFFSET 990 +-- !query analysis +Offset 990 ++- Sort [unique1#x ASC NULLS FIRST], true + +- Project [ AS ten#x, unique1#x, unique2#x, stringu1#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT '' AS five, unique1, unique2, stringu1 + FROM onek + ORDER BY unique1 LIMIT 5 OFFSET 900 +-- !query analysis +GlobalLimit 5 ++- LocalLimit 5 + +- Offset 900 + +- Sort [unique1#x ASC NULLS FIRST], true + +- Project [ AS five#x, unique1#x, unique2#x, stringu1#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2) +-- !query analysis +CreateViewCommand `INT8_TBL`, SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2), false, true, LocalTempView, true + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select * from int8_tbl limit (case when random() < 0.5 then bigint(null) end) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2400", + "messageParameters" : { + "limitExpr" : "CASE WHEN (_nondeterministic < CAST(0.5BD AS DOUBLE)) THEN CAST(NULL AS BIGINT) END", + "name" : "limit" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 31, + "stopIndex" : 76, + "fragment" : "case when random() < 0.5 then bigint(null) end" + } ] +} + + +-- !query +select * from int8_tbl offset (case when random() < 0.5 then bigint(null) end) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2400", + "messageParameters" : { + "limitExpr" : "CASE WHEN (_nondeterministic < CAST(0.5BD AS DOUBLE)) THEN CAST(NULL AS BIGINT) END", + "name" : "offset" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 77, + "fragment" : "case when random() < 0.5 then bigint(null) end" + } ] +} + + +-- !query +DROP VIEW INT8_TBL +-- !query analysis +DropTempViewCommand INT8_TBL + + +-- !query +select sum(tenthous) as s1, sum(tenthous) + random()*0 as s2 + from tenk1 group by thousand order by thousand limit 3 +-- !query analysis +[Analyzer test output redacted due to nondeterminism] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out new file mode 100644 index 0000000000000..7c4ffa767104b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out @@ -0,0 +1,4967 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE num_data (id int, val decimal(38,10)) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`num_data`, false + + +-- !query +CREATE TABLE num_exp_add (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`num_exp_add`, false + + +-- !query +CREATE TABLE num_exp_sub (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`num_exp_sub`, false + + +-- !query +CREATE TABLE num_exp_div (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`num_exp_div`, false + + +-- !query +CREATE TABLE num_exp_mul (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`num_exp_mul`, false + + +-- !query +CREATE TABLE num_exp_sqrt (id int, expected decimal(38,10)) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`num_exp_sqrt`, false + + +-- !query +CREATE TABLE num_exp_ln (id int, expected decimal(38,10)) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`num_exp_ln`, false + + +-- !query +CREATE TABLE num_exp_log10 (id int, expected decimal(38,10)) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`num_exp_log10`, false + + +-- !query +CREATE TABLE num_exp_power_10_ln (id int, expected decimal(38,10)) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`num_exp_power_10_ln`, false + + +-- !query +CREATE TABLE num_result (id1 int, id2 int, result decimal(38,10)) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`num_result`, false + + +-- !query +INSERT INTO num_exp_add VALUES (0,0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (0,0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (0,0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (0,0,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (0,1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (0,1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (0,1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (0,1,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (0,2,-34338492.215397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (0,2,34338492.215397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (0,2,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (0,2,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (0,3,4.31) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (0,3,-4.31) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (0,3,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (0,3,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (0,4,7799461.4119) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (0,4,-7799461.4119) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (0,4,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (0,4,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (0,5,16397.038491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (0,5,-16397.038491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (0,5,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (0,5,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (0,6,93901.57763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (0,6,-93901.57763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (0,6,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (0,6,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (0,7,-83028485) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (0,7,83028485) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (0,7,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (0,7,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (0,8,74881) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (0,8,-74881) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (0,8,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (0,8,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (0,9,-24926804.045047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (0,9,24926804.045047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (0,9,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (0,9,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (1,0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (1,0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (1,0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (1,0,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (1,1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (1,1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (1,1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (1,1,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (1,2,-34338492.215397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (1,2,34338492.215397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (1,2,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (1,2,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (1,3,4.31) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (1,3,-4.31) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (1,3,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (1,3,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (1,4,7799461.4119) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (1,4,-7799461.4119) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (1,4,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (1,4,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (1,5,16397.038491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (1,5,-16397.038491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (1,5,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (1,5,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (1,6,93901.57763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (1,6,-93901.57763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (1,6,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (1,6,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (1,7,-83028485) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (1,7,83028485) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (1,7,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (1,7,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (1,8,74881) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (1,8,-74881) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (1,8,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (1,8,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (1,9,-24926804.045047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (1,9,24926804.045047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (1,9,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (1,9,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (2,0,-34338492.215397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (2,0,-34338492.215397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (2,0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (2,0,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (2,1,-34338492.215397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (2,1,-34338492.215397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (2,1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (2,1,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (2,2,-68676984.430794094) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (2,2,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (2,2,1179132047626883.596862135856320209) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (2,2,1.00000000000000000000) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (2,3,-34338487.905397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (2,3,-34338496.525397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (2,3,-147998901.44836127257) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (2,3,-7967167.56737750510440835266) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (2,4,-26539030.803497047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (2,4,-42137953.627297047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (2,4,-267821744976817.8111137106593) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (2,4,-4.40267480046830116685) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (2,5,-34322095.176906047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (2,5,-34354889.253888047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (2,5,-563049578578.769242506736077) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (2,5,-2094.18866914563535496429) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (2,6,-34244590.637766787) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (2,6,-34432393.793027307) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (2,6,-3224438592470.18449811926184222) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (2,6,-365.68599891479766440940) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (2,7,-117366977.215397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (2,7,48689992.784602953) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (2,7,2851072985828710.485883795) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (2,7,.41357483778485235518) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (2,8,-34263611.215397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (2,8,-34413373.215397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (2,8,-2571300635581.146276407) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (2,8,-458.57416721727870888476) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (2,9,-59265296.260444467) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (2,9,-9411688.170349627) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (2,9,855948866655588.453741509242968740) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (2,9,1.37757299946438931811) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (3,0,4.31) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (3,0,4.31) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (3,0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (3,0,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (3,1,4.31) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (3,1,4.31) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (3,1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (3,1,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (3,2,-34338487.905397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (3,2,34338496.525397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (3,2,-147998901.44836127257) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (3,2,-.00000012551512084352) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (3,3,8.62) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (3,3,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (3,3,18.5761) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (3,3,1.00000000000000000000) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (3,4,7799465.7219) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (3,4,-7799457.1019) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (3,4,33615678.685289) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (3,4,.00000055260225961552) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (3,5,16401.348491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (3,5,-16392.728491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (3,5,70671.23589621) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (3,5,.00026285234387695504) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (3,6,93905.88763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (3,6,-93897.26763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (3,6,404715.7995864206) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (3,6,.00004589912234457595) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (3,7,-83028480.69) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (3,7,83028489.31) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (3,7,-357852770.35) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (3,7,-.00000005190989574240) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (3,8,74885.31) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (3,8,-74876.69) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (3,8,322737.11) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (3,8,.00005755799201399553) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (3,9,-24926799.735047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (3,9,24926808.355047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (3,9,-107434525.43415438020) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (3,9,-.00000017290624149854) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (4,0,7799461.4119) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (4,0,7799461.4119) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (4,0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (4,0,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (4,1,7799461.4119) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (4,1,7799461.4119) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (4,1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (4,1,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (4,2,-26539030.803497047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (4,2,42137953.627297047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (4,2,-267821744976817.8111137106593) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (4,2,-.22713465002993920385) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (4,3,7799465.7219) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (4,3,7799457.1019) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (4,3,33615678.685289) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (4,3,1809619.81714617169373549883) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (4,4,15598922.8238) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (4,4,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (4,4,60831598315717.14146161) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (4,4,1.00000000000000000000) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (4,5,7815858.450391) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (4,5,7783064.373409) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (4,5,127888068979.9935054429) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (4,5,475.66281046305802686061) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (4,6,7893362.98953026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (4,6,7705559.83426974) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (4,6,732381731243.745115764094) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (4,6,83.05996138436129499606) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (4,7,-75229023.5881) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (4,7,90827946.4119) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (4,7,-647577464846017.9715) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (4,7,-.09393717604145131637) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (4,8,7874342.4119) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (4,8,7724580.4119) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (4,8,584031469984.4839) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (4,8,104.15808298366741897143) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (4,9,-17127342.633147420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (4,9,32726265.456947420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (4,9,-194415646271340.1815956522980) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (4,9,-.31289456112403769409) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (5,0,16397.038491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (5,0,16397.038491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (5,0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (5,0,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (5,1,16397.038491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (5,1,16397.038491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (5,1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (5,1,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (5,2,-34322095.176906047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (5,2,34354889.253888047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (5,2,-563049578578.769242506736077) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (5,2,-.00047751189505192446) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (5,3,16401.348491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (5,3,16392.728491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (5,3,70671.23589621) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (5,3,3804.41728329466357308584) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (5,4,7815858.450391) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (5,4,-7783064.373409) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (5,4,127888068979.9935054429) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (5,4,.00210232958726897192) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (5,5,32794.076982) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (5,5,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (5,5,268862871.275335557081) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (5,5,1.00000000000000000000) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (5,6,110298.61612126) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (5,6,-77504.53913926) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (5,6,1539707782.76899778633766) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (5,6,.17461941433576102689) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (5,7,-83012087.961509) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (5,7,83044882.038491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (5,7,-1361421264394.416135) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (5,7,-.00019748690453643710) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (5,8,91278.038491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (5,8,-58483.961509) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (5,8,1227826639.244571) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (5,8,.21897461960978085228) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (5,9,-24910407.006556420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (5,9,24943201.083538420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (5,9,-408725765384.257043660243220) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (5,9,-.00065780749354660427) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (6,0,93901.57763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (6,0,93901.57763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (6,0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (6,0,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (6,1,93901.57763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (6,1,93901.57763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (6,1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (6,1,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (6,2,-34244590.637766787) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (6,2,34432393.793027307) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (6,2,-3224438592470.18449811926184222) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (6,2,-.00273458651128995823) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (6,3,93905.88763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (6,3,93897.26763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (6,3,404715.7995864206) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (6,3,21786.90896293735498839907) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (6,4,7893362.98953026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (6,4,-7705559.83426974) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (6,4,732381731243.745115764094) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (6,4,.01203949512295682469) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (6,5,110298.61612126) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (6,5,77504.53913926) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (6,5,1539707782.76899778633766) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (6,5,5.72674008674192359679) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (6,6,187803.15526052) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (6,6,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (6,6,8817506281.4517452372676676) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (6,6,1.00000000000000000000) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (6,7,-82934583.42236974) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (6,7,83122386.57763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (6,7,-7796505729750.37795610) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (6,7,-.00113095617281538980) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (6,8,168782.57763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (6,8,19020.57763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (6,8,7031444034.53149906) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (6,8,1.25401073209839612184) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (6,9,-24832902.467417160) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (6,9,25020705.622677680) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (6,9,-2340666225110.29929521292692920) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (6,9,-.00376709254265256789) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (7,0,-83028485) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (7,0,-83028485) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (7,0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (7,0,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (7,1,-83028485) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (7,1,-83028485) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (7,1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (7,1,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (7,2,-117366977.215397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (7,2,-48689992.784602953) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (7,2,2851072985828710.485883795) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (7,2,2.41794207151503385700) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (7,3,-83028480.69) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (7,3,-83028489.31) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (7,3,-357852770.35) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (7,3,-19264149.65197215777262180974) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (7,4,-75229023.5881) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (7,4,-90827946.4119) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (7,4,-647577464846017.9715) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (7,4,-10.64541262725136247686) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (7,5,-83012087.961509) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (7,5,-83044882.038491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (7,5,-1361421264394.416135) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (7,5,-5063.62688881730941836574) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (7,6,-82934583.42236974) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (7,6,-83122386.57763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (7,6,-7796505729750.37795610) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (7,6,-884.20756174009028770294) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (7,7,-166056970) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (7,7,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (7,7,6893729321395225) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#xL as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#xL] + + +-- !query +INSERT INTO num_exp_div VALUES (7,7,1.00000000000000000000) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (7,8,-82953604) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (7,8,-83103366) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (7,8,-6217255985285) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#xL as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#xL] + + +-- !query +INSERT INTO num_exp_div VALUES (7,8,-1108.80577182462841041118) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (7,9,-107955289.045047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (7,9,-58101680.954952580) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (7,9,2069634775752159.035758700) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (7,9,3.33089171198810413382) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (8,0,74881) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (8,0,74881) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (8,0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (8,0,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (8,1,74881) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (8,1,74881) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (8,1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (8,1,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (8,2,-34263611.215397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (8,2,34413373.215397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (8,2,-2571300635581.146276407) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (8,2,-.00218067233500788615) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (8,3,74885.31) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (8,3,74876.69) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (8,3,322737.11) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (8,3,17373.78190255220417633410) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (8,4,7874342.4119) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (8,4,-7724580.4119) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (8,4,584031469984.4839) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (8,4,.00960079113741758956) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (8,5,91278.038491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (8,5,58483.961509) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (8,5,1227826639.244571) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (8,5,4.56673929509287019456) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (8,6,168782.57763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (8,6,-19020.57763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (8,6,7031444034.53149906) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (8,6,.79744134113322314424) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (8,7,-82953604) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (8,7,83103366) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (8,7,-6217255985285) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#xL as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#xL] + + +-- !query +INSERT INTO num_exp_div VALUES (8,7,-.00090187120721280172) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (8,8,149762) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (8,8,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (8,8,5607164161) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#xL as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#xL] + + +-- !query +INSERT INTO num_exp_div VALUES (8,8,1.00000000000000000000) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (8,9,-24851923.045047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (8,9,25001685.045047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (8,9,-1866544013697.195857020) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (8,9,-.00300403532938582735) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (9,0,-24926804.045047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (9,0,-24926804.045047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (9,0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (9,0,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (9,1,-24926804.045047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (9,1,-24926804.045047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (9,1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (9,1,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (9,2,-59265296.260444467) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (9,2,9411688.170349627) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (9,2,855948866655588.453741509242968740) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (9,2,.72591434384152961526) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (9,3,-24926799.735047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (9,3,-24926808.355047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (9,3,-107434525.43415438020) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (9,3,-5783481.21694835730858468677) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (9,4,-17127342.633147420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (9,4,-32726265.456947420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (9,4,-194415646271340.1815956522980) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (9,4,-3.19596478892958416484) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (9,5,-24910407.006556420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (9,5,-24943201.083538420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (9,5,-408725765384.257043660243220) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (9,5,-1520.20159364322004505807) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (9,6,-24832902.467417160) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (9,6,-25020705.622677680) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (9,6,-2340666225110.29929521292692920) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (9,6,-265.45671195426965751280) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (9,7,-107955289.045047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (9,7,58101680.954952580) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (9,7,2069634775752159.035758700) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (9,7,.30021990699995814689) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (9,8,-24851923.045047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (9,8,-25001685.045047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (9,8,-1866544013697.195857020) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (9,8,-332.88556569820675471748) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_add VALUES (9,9,-49853608.090094840) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sub VALUES (9,9,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_mul VALUES (9,9,621345559900192.420120630048656400) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_div VALUES (9,9,1.00000000000000000000) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO num_exp_sqrt VALUES (0,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_sqrt VALUES (1,0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_sqrt VALUES (2,5859.90547836712524903505) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_sqrt VALUES (3,2.07605394920266944396) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_sqrt VALUES (4,2792.75158435189147418923) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_sqrt VALUES (5,128.05092147657509145473) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_sqrt VALUES (6,306.43364311096782703406) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_sqrt VALUES (7,9111.99676251039939975230) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_sqrt VALUES (8,273.64392922189960397542) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_sqrt VALUES (9,4992.67503899937593364766) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_ln VALUES (0,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_ln VALUES (1,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_ln VALUES (2,17.35177750493897715514) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_ln VALUES (3,1.46093790411565641971) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_ln VALUES (4,15.86956523951936572464) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_ln VALUES (5,9.70485601768871834038) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_ln VALUES (6,11.45000246622944403127) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_ln VALUES (7,18.23469429965478772991) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_ln VALUES (8,11.22365546576315513668) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_ln VALUES (9,17.03145425013166006962) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_log10 VALUES (0,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_log10 VALUES (1,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_log10 VALUES (2,7.53578122160797276459) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_log10 VALUES (3,.63447727016073160075) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_log10 VALUES (4,6.89206461372691743345) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_log10 VALUES (5,4.21476541614777768626) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_log10 VALUES (6,4.97267288886207207671) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_log10 VALUES (7,7.91922711353275546914) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_log10 VALUES (8,4.87437163556421004138) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_log10 VALUES (9,7.39666659961986567059) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (0,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (1,double('NaN')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (2,224790267919917955.13261618583642653184) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (3,28.90266599445155957393) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (4,7405685069594999.07733999469386277636) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (5,5068226527.32127265408584640098) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (6,281839893606.99372343357047819067) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (7,1716699575118597095.42330819910640247627) +-- !query analysis +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", + "sqlState" : "22003", + "messageParameters" : { + "maxPrecision" : "38", + "precision" : "39" + } +} + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (8,167361463828.07491320069016125952) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (9,107511333880052007.04141124673540337457) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_data VALUES (0, 0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_data VALUES (1, 0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_data VALUES (2, -34338492.215397047) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_data VALUES (3, 4.31) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_data VALUES (4, 7799461.4119) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_data VALUES (5, 16397.038491) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_data VALUES (6, 93901.57763026) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_data VALUES (7, -83028485) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_data VALUES (8, 74881) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO num_data VALUES (9, -24926804.045047420) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM num_data +-- !query analysis +Project [id#x, val#x] ++- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +TRUNCATE TABLE num_result +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`num_result` + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, t1.val + t2.val + FROM num_data t1, num_data t2 +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result] ++- Project [cast(id#x as int) AS id1#x, cast(id#x as int) AS id2#x, cast((val + val)#x as decimal(38,10)) AS result#x] + +- Project [id#x, id#x, (val#x + val#x) AS (val + val)#x] + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_data + : +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query analysis +Project [id1#x, id2#x, result#x, expected#x] ++- Filter (((id1#x = id1#x) AND (id2#x = id2#x)) AND NOT (result#x = expected#x)) + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_result + : +- Relation spark_catalog.default.num_result[id1#x,id2#x,result#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_exp_add + +- Relation spark_catalog.default.num_exp_add[id1#x,id2#x,expected#x] parquet + + +-- !query +TRUNCATE TABLE num_result +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`num_result` + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val + t2.val, 10) + FROM num_data t1, num_data t2 +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result] ++- Project [cast(id#x as int) AS id1#x, cast(id#x as int) AS id2#x, cast(round((val + val), 10)#x as decimal(38,10)) AS result#x] + +- Project [id#x, id#x, round((val#x + val#x), 10) AS round((val + val), 10)#x] + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_data + : +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 10) as expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 10) +-- !query analysis +Project [id1#x, id2#x, result#x, round(expected#x, 10) AS expected#x] ++- Filter (((id1#x = id1#x) AND (id2#x = id2#x)) AND NOT (result#x = round(expected#x, 10))) + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_result + : +- Relation spark_catalog.default.num_result[id1#x,id2#x,result#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_exp_add + +- Relation spark_catalog.default.num_exp_add[id1#x,id2#x,expected#x] parquet + + +-- !query +TRUNCATE TABLE num_result +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`num_result` + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, t1.val - t2.val + FROM num_data t1, num_data t2 +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result] ++- Project [cast(id#x as int) AS id1#x, cast(id#x as int) AS id2#x, cast((val - val)#x as decimal(38,10)) AS result#x] + +- Project [id#x, id#x, (val#x - val#x) AS (val - val)#x] + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_data + : +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query analysis +Project [id1#x, id2#x, result#x, expected#x] ++- Filter (((id1#x = id1#x) AND (id2#x = id2#x)) AND NOT (result#x = expected#x)) + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_result + : +- Relation spark_catalog.default.num_result[id1#x,id2#x,result#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_exp_sub + +- Relation spark_catalog.default.num_exp_sub[id1#x,id2#x,expected#x] parquet + + +-- !query +TRUNCATE TABLE num_result +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`num_result` + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val - t2.val, 40) + FROM num_data t1, num_data t2 +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result] ++- Project [cast(id#x as int) AS id1#x, cast(id#x as int) AS id2#x, cast(round((val - val), 40)#x as decimal(38,10)) AS result#x] + +- Project [id#x, id#x, round((val#x - val#x), 40) AS round((val - val), 40)#x] + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_data + : +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 40) + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 40) +-- !query analysis +Project [id1#x, id2#x, result#x, round(expected#x, 40) AS round(expected, 40)#x] ++- Filter (((id1#x = id1#x) AND (id2#x = id2#x)) AND NOT (result#x = round(expected#x, 40))) + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_result + : +- Relation spark_catalog.default.num_result[id1#x,id2#x,result#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_exp_sub + +- Relation spark_catalog.default.num_exp_sub[id1#x,id2#x,expected#x] parquet + + +-- !query +TRUNCATE TABLE num_result +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`num_result` + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, t1.val, t2.val, t1.val * t2.val + FROM num_data t1, num_data t2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1168", + "messageParameters" : { + "insertedColumns" : "5", + "staticPartCols" : "0", + "tableName" : "`spark_catalog`.`default`.`num_result`", + "targetColumns" : "3" + } +} + + +-- !query +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query analysis +Project [id1#x, id2#x, result#x, expected#x] ++- Filter (((id1#x = id1#x) AND (id2#x = id2#x)) AND NOT (result#x = expected#x)) + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_result + : +- Relation spark_catalog.default.num_result[id1#x,id2#x,result#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_exp_mul + +- Relation spark_catalog.default.num_exp_mul[id1#x,id2#x,expected#x] parquet + + +-- !query +TRUNCATE TABLE num_result +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`num_result` + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val * t2.val, 30) + FROM num_data t1, num_data t2 +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result] ++- Project [cast(id#x as int) AS id1#x, cast(id#x as int) AS id2#x, cast(round((val * val), 30)#x as decimal(38,10)) AS result#x] + +- Project [id#x, id#x, round((val#x * val#x), 30) AS round((val * val), 30)#x] + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_data + : +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 30) as expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 30) +-- !query analysis +Project [id1#x, id2#x, result#x, round(expected#x, 30) AS expected#x] ++- Filter (((id1#x = id1#x) AND (id2#x = id2#x)) AND NOT (result#x = round(expected#x, 30))) + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_result + : +- Relation spark_catalog.default.num_result[id1#x,id2#x,result#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_exp_mul + +- Relation spark_catalog.default.num_exp_mul[id1#x,id2#x,expected#x] parquet + + +-- !query +TRUNCATE TABLE num_result +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`num_result` + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, t1.val / t2.val + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0' +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result] ++- Project [cast(id#x as int) AS id1#x, cast(id#x as int) AS id2#x, cast((val / val)#x as decimal(38,10)) AS result#x] + +- Project [id#x, id#x, (val#x / val#x) AS (val / val)#x] + +- Filter NOT (cast(val#x as double) = cast(0.0 as double)) + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_data + : +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query analysis +Project [id1#x, id2#x, result#x, expected#x] ++- Filter (((id1#x = id1#x) AND (id2#x = id2#x)) AND NOT (result#x = expected#x)) + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_result + : +- Relation spark_catalog.default.num_result[id1#x,id2#x,result#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_exp_div + +- Relation spark_catalog.default.num_exp_div[id1#x,id2#x,expected#x] parquet + + +-- !query +TRUNCATE TABLE num_result +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`num_result` + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val / t2.val, 80) + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0' +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result] ++- Project [cast(id#x as int) AS id1#x, cast(id#x as int) AS id2#x, cast(round((val / val), 80)#x as decimal(38,10)) AS result#x] + +- Project [id#x, id#x, round((val#x / val#x), 80) AS round((val / val), 80)#x] + +- Filter NOT (cast(val#x as double) = cast(0.0 as double)) + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_data + : +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 80) as expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 80) +-- !query analysis +Project [id1#x, id2#x, result#x, round(expected#x, 80) AS expected#x] ++- Filter (((id1#x = id1#x) AND (id2#x = id2#x)) AND NOT (result#x = round(expected#x, 80))) + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_result + : +- Relation spark_catalog.default.num_result[id1#x,id2#x,result#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_exp_div + +- Relation spark_catalog.default.num_exp_div[id1#x,id2#x,expected#x] parquet + + +-- !query +TRUNCATE TABLE num_result +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`num_result` + + +-- !query +INSERT INTO num_result SELECT id, 0, SQRT(ABS(val)) + FROM num_data +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result] ++- Project [cast(id#x as int) AS id1#x, cast(0#x as int) AS id2#x, cast(SQRT(abs(val))#x as decimal(38,10)) AS result#x] + +- Project [id#x, 0 AS 0#x, SQRT(cast(abs(val#x) as double)) AS SQRT(abs(val))#x] + +- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_sqrt t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query analysis +Project [id1#x, result#x, expected#x] ++- Filter ((id1#x = id#x) AND NOT (result#x = expected#x)) + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_result + : +- Relation spark_catalog.default.num_result[id1#x,id2#x,result#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_exp_sqrt + +- Relation spark_catalog.default.num_exp_sqrt[id#x,expected#x] parquet + + +-- !query +TRUNCATE TABLE num_result +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`num_result` + + +-- !query +INSERT INTO num_result SELECT id, 0, LN(ABS(val)) + FROM num_data + WHERE val != '0.0' +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result] ++- Project [cast(id#x as int) AS id1#x, cast(0#x as int) AS id2#x, cast(ln(abs(val))#x as decimal(38,10)) AS result#x] + +- Project [id#x, 0 AS 0#x, ln(cast(abs(val#x) as double)) AS ln(abs(val))#x] + +- Filter NOT (cast(val#x as double) = cast(0.0 as double)) + +- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query analysis +Project [id1#x, result#x, expected#x] ++- Filter ((id1#x = id#x) AND NOT (result#x = expected#x)) + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_result + : +- Relation spark_catalog.default.num_result[id1#x,id2#x,result#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_exp_ln + +- Relation spark_catalog.default.num_exp_ln[id#x,expected#x] parquet + + +-- !query +TRUNCATE TABLE num_result +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`num_result` + + +-- !query +INSERT INTO num_result SELECT id, 0, LOG(cast('10' as decimal(38, 18)), ABS(val)) + FROM num_data + WHERE val != '0.0' +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result] ++- Project [cast(id#x as int) AS id1#x, cast(0#x as int) AS id2#x, cast(LOG(CAST(10 AS DECIMAL(38,18)), abs(val))#x as decimal(38,10)) AS result#x] + +- Project [id#x, 0 AS 0#x, LOG(cast(cast(10 as decimal(38,18)) as double), cast(abs(val#x) as double)) AS LOG(CAST(10 AS DECIMAL(38,18)), abs(val))#x] + +- Filter NOT (cast(val#x as double) = cast(0.0 as double)) + +- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_log10 t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query analysis +Project [id1#x, result#x, expected#x] ++- Filter ((id1#x = id#x) AND NOT (result#x = expected#x)) + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_result + : +- Relation spark_catalog.default.num_result[id1#x,id2#x,result#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_exp_log10 + +- Relation spark_catalog.default.num_exp_log10[id#x,expected#x] parquet + + +-- !query +TRUNCATE TABLE num_result +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`num_result` + + +-- !query +INSERT INTO num_result SELECT id, 0, POWER(cast('10' as decimal(38, 18)), LN(ABS(round(val,200)))) + FROM num_data + WHERE val != '0.0' +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result] ++- Project [cast(id#x as int) AS id1#x, cast(0#x as int) AS id2#x, cast(POWER(CAST(10 AS DECIMAL(38,18)), ln(abs(round(val, 200))))#x as decimal(38,10)) AS result#x] + +- Project [id#x, 0 AS 0#x, POWER(cast(cast(10 as decimal(38,18)) as double), ln(cast(abs(round(val#x, 200)) as double))) AS POWER(CAST(10 AS DECIMAL(38,18)), ln(abs(round(val, 200))))#x] + +- Filter NOT (cast(val#x as double) = cast(0.0 as double)) + +- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_power_10_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query analysis +Project [id1#x, result#x, expected#x] ++- Filter ((id1#x = id#x) AND NOT (result#x = expected#x)) + +- Join Inner + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.num_result + : +- Relation spark_catalog.default.num_result[id1#x,id2#x,result#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.num_exp_power_10_ln + +- Relation spark_catalog.default.num_exp_power_10_ln[id#x,expected#x] parquet + + +-- !query +SELECT AVG(val) FROM num_data +-- !query analysis +Aggregate [avg(val#x) AS avg(val)#x] ++- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +CREATE TABLE fract_only (id int, val decimal(4,4)) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`fract_only`, false + + +-- !query +INSERT INTO fract_only VALUES (1, 0.0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/fract_only, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/fract_only], Append, `spark_catalog`.`default`.`fract_only`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/fract_only), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(4,4)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO fract_only VALUES (2, 0.1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/fract_only, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/fract_only], Append, `spark_catalog`.`default`.`fract_only`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/fract_only), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(4,4)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO fract_only VALUES (4, -0.9999) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/fract_only, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/fract_only], Append, `spark_catalog`.`default`.`fract_only`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/fract_only), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(4,4)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO fract_only VALUES (5, 0.99994) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/fract_only, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/fract_only], Append, `spark_catalog`.`default`.`fract_only`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/fract_only), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(4,4)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO fract_only VALUES (7, 0.00001) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/fract_only, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/fract_only], Append, `spark_catalog`.`default`.`fract_only`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/fract_only), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(4,4)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO fract_only VALUES (8, 0.00017) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/fract_only, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/fract_only], Append, `spark_catalog`.`default`.`fract_only`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/fract_only), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(4,4)) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM fract_only +-- !query analysis +Project [id#x, val#x] ++- SubqueryAlias spark_catalog.default.fract_only + +- Relation spark_catalog.default.fract_only[id#x,val#x] parquet + + +-- !query +DROP TABLE fract_only +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`fract_only`, false, false, false + + +-- !query +SELECT decimal(double('NaN')) +-- !query analysis +Project [cast(cast(NaN as double) as decimal(10,0)) AS NaN#x] ++- OneRowRelation + + +-- !query +SELECT decimal(double('Infinity')) +-- !query analysis +Project [cast(cast(Infinity as double) as decimal(10,0)) AS Infinity#x] ++- OneRowRelation + + +-- !query +SELECT decimal(double('-Infinity')) +-- !query analysis +Project [cast(cast(-Infinity as double) as decimal(10,0)) AS -Infinity#x] ++- OneRowRelation + + +-- !query +SELECT decimal(float('NaN')) +-- !query analysis +Project [cast(cast(NaN as float) as decimal(10,0)) AS NaN#x] ++- OneRowRelation + + +-- !query +SELECT decimal(float('Infinity')) +-- !query analysis +Project [cast(cast(Infinity as float) as decimal(10,0)) AS Infinity#x] ++- OneRowRelation + + +-- !query +SELECT decimal(float('-Infinity')) +-- !query analysis +Project [cast(cast(-Infinity as float) as decimal(10,0)) AS -Infinity#x] ++- OneRowRelation + + +-- !query +CREATE TABLE ceil_floor_round (a decimal(38, 18)) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`ceil_floor_round`, false + + +-- !query +INSERT INTO ceil_floor_round VALUES (-5.5) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/ceil_floor_round, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/ceil_floor_round], Append, `spark_catalog`.`default`.`ceil_floor_round`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/ceil_floor_round), [a] ++- Project [cast(col1#x as decimal(38,18)) AS a#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO ceil_floor_round VALUES (-5.499999) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/ceil_floor_round, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/ceil_floor_round], Append, `spark_catalog`.`default`.`ceil_floor_round`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/ceil_floor_round), [a] ++- Project [cast(col1#x as decimal(38,18)) AS a#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO ceil_floor_round VALUES (9.5) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/ceil_floor_round, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/ceil_floor_round], Append, `spark_catalog`.`default`.`ceil_floor_round`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/ceil_floor_round), [a] ++- Project [cast(col1#x as decimal(38,18)) AS a#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO ceil_floor_round VALUES (9.4999999) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/ceil_floor_round, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/ceil_floor_round], Append, `spark_catalog`.`default`.`ceil_floor_round`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/ceil_floor_round), [a] ++- Project [cast(col1#x as decimal(38,18)) AS a#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO ceil_floor_round VALUES (0.0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/ceil_floor_round, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/ceil_floor_round], Append, `spark_catalog`.`default`.`ceil_floor_round`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/ceil_floor_round), [a] ++- Project [cast(col1#x as decimal(38,18)) AS a#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO ceil_floor_round VALUES (0.0000001) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/ceil_floor_round, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/ceil_floor_round], Append, `spark_catalog`.`default`.`ceil_floor_round`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/ceil_floor_round), [a] ++- Project [cast(col1#x as decimal(38,18)) AS a#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO ceil_floor_round VALUES (-0.000001) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/ceil_floor_round, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/ceil_floor_round], Append, `spark_catalog`.`default`.`ceil_floor_round`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/ceil_floor_round), [a] ++- Project [cast(col1#x as decimal(38,18)) AS a#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round +-- !query analysis +Project [a#x, CEIL(a#x) AS CEIL(a)#x, ceiling(a#x) AS ceiling(a)#x, FLOOR(a#x) AS FLOOR(a)#x, round(a#x, 0) AS round(a, 0)#x] ++- SubqueryAlias spark_catalog.default.ceil_floor_round + +- Relation spark_catalog.default.ceil_floor_round[a#x] parquet + + +-- !query +DROP TABLE ceil_floor_round +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`ceil_floor_round`, false, false, false + + +-- !query +SELECT width_bucket(5.0, 3.0, 4.0, 0) +-- !query analysis +Project [width_bucket(cast(5.0 as double), cast(3.0 as double), cast(4.0 as double), cast(0 as bigint)) AS width_bucket(5.0, 3.0, 4.0, 0)#xL] ++- OneRowRelation + + +-- !query +SELECT width_bucket(5.0, 3.0, 4.0, -5) +-- !query analysis +Project [width_bucket(cast(5.0 as double), cast(3.0 as double), cast(4.0 as double), cast(-5 as bigint)) AS width_bucket(5.0, 3.0, 4.0, -5)#xL] ++- OneRowRelation + + +-- !query +SELECT width_bucket(3.5, 3.0, 3.0, 888) +-- !query analysis +Project [width_bucket(cast(3.5 as double), cast(3.0 as double), cast(3.0 as double), cast(888 as bigint)) AS width_bucket(3.5, 3.0, 3.0, 888)#xL] ++- OneRowRelation + + +-- !query +SELECT width_bucket(double(5.0), double(3.0), double(4.0), 0) +-- !query analysis +Project [width_bucket(cast(5.0 as double), cast(3.0 as double), cast(4.0 as double), cast(0 as bigint)) AS width_bucket(5.0, 3.0, 4.0, 0)#xL] ++- OneRowRelation + + +-- !query +SELECT width_bucket(double(5.0), double(3.0), double(4.0), -5) +-- !query analysis +Project [width_bucket(cast(5.0 as double), cast(3.0 as double), cast(4.0 as double), cast(-5 as bigint)) AS width_bucket(5.0, 3.0, 4.0, -5)#xL] ++- OneRowRelation + + +-- !query +SELECT width_bucket(double(3.5), double(3.0), double(3.0), 888) +-- !query analysis +Project [width_bucket(cast(3.5 as double), cast(3.0 as double), cast(3.0 as double), cast(888 as bigint)) AS width_bucket(3.5, 3.0, 3.0, 888)#xL] ++- OneRowRelation + + +-- !query +SELECT width_bucket('NaN', 3.0, 4.0, 888) +-- !query analysis +Project [width_bucket(cast(NaN as double), cast(3.0 as double), cast(4.0 as double), cast(888 as bigint)) AS width_bucket(NaN, 3.0, 4.0, 888)#xL] ++- OneRowRelation + + +-- !query +SELECT width_bucket(double(0), 'NaN', double(4.0), 888) +-- !query analysis +Project [width_bucket(cast(0 as double), cast(NaN as double), cast(4.0 as double), cast(888 as bigint)) AS width_bucket(0, NaN, 4.0, 888)#xL] ++- OneRowRelation + + +-- !query +CREATE TABLE width_bucket_test (operand_num decimal(30,15), operand_f8 double) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`width_bucket_test`, false + + +-- !query +INSERT INTO width_bucket_test VALUES + (-5.2, -5.2), + (-0.0000000001, -0.0000000001), + (0.000000000001, 0.000000000001), + (1, 1), + (1.99999999999999, 1.99999999999999), + (2, 2), + (2.00000000000001, 2.00000000000001), + (3, 3), + (4, 4), + (4.5, 4.5), + (5, 5), + (5.5, 5.5), + (6, 6), + (7, 7), + (8, 8), + (9, 9), + (9.99999999999999, 9.99999999999999), + (10, 10), + (10.0000000000001, 10.0000000000001) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/width_bucket_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/width_bucket_test], Append, `spark_catalog`.`default`.`width_bucket_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/width_bucket_test), [operand_num, operand_f8] ++- Project [cast(col1#x as decimal(30,15)) AS operand_num#x, cast(col2#x as double) AS operand_f8#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT + operand_num, + width_bucket(operand_num, 0, 10, 5) AS wb_1, + width_bucket(operand_f8, 0, 10, 5) AS wb_1f, + width_bucket(operand_num, 10, 0, 5) AS wb_2, + width_bucket(operand_f8, 10, 0, 5) AS wb_2f, + width_bucket(operand_num, 2, 8, 4) AS wb_3, + width_bucket(operand_f8, 2, 8, 4) AS wb_3f, + width_bucket(operand_num, 5.0, 5.5, 20) AS wb_4, + width_bucket(operand_f8, 5.0, 5.5, 20) AS wb_4f, + width_bucket(operand_num, -25, 25, 10) AS wb_5, + width_bucket(operand_f8, -25, 25, 10) AS wb_5f + FROM width_bucket_test + ORDER BY operand_num ASC +-- !query analysis +Sort [operand_num#x ASC NULLS FIRST], true ++- Project [operand_num#x, width_bucket(cast(operand_num#x as double), cast(0 as double), cast(10 as double), cast(5 as bigint)) AS wb_1#xL, width_bucket(operand_f8#x, cast(0 as double), cast(10 as double), cast(5 as bigint)) AS wb_1f#xL, width_bucket(cast(operand_num#x as double), cast(10 as double), cast(0 as double), cast(5 as bigint)) AS wb_2#xL, width_bucket(operand_f8#x, cast(10 as double), cast(0 as double), cast(5 as bigint)) AS wb_2f#xL, width_bucket(cast(operand_num#x as double), cast(2 as double), cast(8 as double), cast(4 as bigint)) AS wb_3#xL, width_bucket(operand_f8#x, cast(2 as double), cast(8 as double), cast(4 as bigint)) AS wb_3f#xL, width_bucket(cast(operand_num#x as double), cast(5.0 as double), cast(5.5 as double), cast(20 as bigint)) AS wb_4#xL, width_bucket(operand_f8#x, cast(5.0 as double), cast(5.5 as double), cast(20 as bigint)) AS wb_4f#xL, width_bucket(cast(operand_num#x as double), cast(-25 as double), cast(25 as double), cast(10 as bigint)) AS wb_5#xL, width_bucket(operand_f8#x, cast(-25 as double), cast(25 as double), cast(10 as bigint)) AS wb_5f#xL] + +- SubqueryAlias spark_catalog.default.width_bucket_test + +- Relation spark_catalog.default.width_bucket_test[operand_num#x,operand_f8#x] parquet + + +-- !query +SELECT width_bucket(double(0.0), double('Infinity'), 5, 10) +-- !query analysis +Project [width_bucket(cast(0.0 as double), cast(Infinity as double), cast(5 as double), cast(10 as bigint)) AS width_bucket(0.0, Infinity, 5, 10)#xL] ++- OneRowRelation + + +-- !query +SELECT width_bucket(double(0.0), 5, double('-Infinity'), 20) +-- !query analysis +Project [width_bucket(cast(0.0 as double), cast(5 as double), cast(-Infinity as double), cast(20 as bigint)) AS width_bucket(0.0, 5, -Infinity, 20)#xL] ++- OneRowRelation + + +-- !query +SELECT width_bucket(double('Infinity'), 1, 10, 10), + width_bucket(double('-Infinity'), 1, 10, 10) +-- !query analysis +Project [width_bucket(cast(Infinity as double), cast(1 as double), cast(10 as double), cast(10 as bigint)) AS width_bucket(Infinity, 1, 10, 10)#xL, width_bucket(cast(-Infinity as double), cast(1 as double), cast(10 as double), cast(10 as bigint)) AS width_bucket(-Infinity, 1, 10, 10)#xL] ++- OneRowRelation + + +-- !query +DROP TABLE width_bucket_test +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`width_bucket_test`, false, false, false + + +-- !query +SELECT '' AS to_char_3, to_char(val, '9999999999999999.999999999999999PR'), val +FROM num_data +-- !query analysis +Project [ AS to_char_3#x, to_char(val#x, 9999999999999999.999999999999999PR) AS to_char(val, 9999999999999999.999999999999999PR)#x, val#x] ++- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT '' AS to_char_4, to_char(val, '9999999999999999.999999999999999S'), val +FROM num_data +-- !query analysis +Project [ AS to_char_4#x, to_char(val#x, 9999999999999999.999999999999999S) AS to_char(val, 9999999999999999.999999999999999S)#x, val#x] ++- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT '' AS to_char_5, to_char(val, 'MI9999999999999999.999999999999999'), val FROM num_data +-- !query analysis +Project [ AS to_char_5#x, to_char(val#x, MI9999999999999999.999999999999999) AS to_char(val, MI9999999999999999.999999999999999)#x, val#x] ++- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT '' AS to_char_9, to_char(val, '0999999999999999.999999999999999'), val FROM num_data +-- !query analysis +Project [ AS to_char_9#x, to_char(val#x, 0999999999999999.999999999999999) AS to_char(val, 0999999999999999.999999999999999)#x, val#x] ++- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT '' AS to_char_10, to_char(val, 'S0999999999999999.999999999999999'), val FROM num_data +-- !query analysis +Project [ AS to_char_10#x, to_char(val#x, S0999999999999999.999999999999999) AS to_char(val, S0999999999999999.999999999999999)#x, val#x] ++- SubqueryAlias spark_catalog.default.num_data + +- Relation spark_catalog.default.num_data[id#x,val#x] parquet + + +-- !query +SELECT '' AS to_number_1, to_number('-34,338,492', '99G999G999') +-- !query analysis +Project [ AS to_number_1#x, to_number(-34,338,492, 99G999G999) AS to_number(-34,338,492, 99G999G999)#x] ++- OneRowRelation + + +-- !query +SELECT '' AS to_number_2, to_number('-34,338,492.654,878', '99G999G999D999G999') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_FORMAT.THOUSANDS_SEPS_MUST_BEFORE_DEC", + "sqlState" : "42601", + "messageParameters" : { + "format" : "'99G999G999D999G999'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 81, + "fragment" : "to_number('-34,338,492.654,878', '99G999G999D999G999')" + } ] +} + + +-- !query +SELECT '' AS to_number_4, to_number('0.00001-', '9.999999S') +-- !query analysis +Project [ AS to_number_4#x, to_number(0.00001-, 9.999999S) AS to_number(0.00001-, 9.999999S)#x] ++- OneRowRelation + + +-- !query +SELECT '' AS to_number_9, to_number('.0', '99999999.99999999') +-- !query analysis +Project [ AS to_number_9#x, to_number(.0, 99999999.99999999) AS to_number(.0, 99999999.99999999)#x] ++- OneRowRelation + + +-- !query +SELECT '' AS to_number_10, to_number('0', '99.99') +-- !query analysis +Project [ AS to_number_10#x, to_number(0, 99.99) AS to_number(0, 99.99)#x] ++- OneRowRelation + + +-- !query +SELECT '' AS to_number_12, to_number('.01-', '99.99S') +-- !query analysis +Project [ AS to_number_12#x, to_number(.01-, 99.99S) AS to_number(.01-, 99.99S)#x] ++- OneRowRelation + + +-- !query +SELECT '' AS to_number_14, to_number('34,50','999,99') +-- !query analysis +Project [ AS to_number_14#x, to_number(34,50, 999,99) AS to_number(34,50, 999,99)#x] ++- OneRowRelation + + +-- !query +SELECT '' AS to_number_15, to_number('123,000','999G') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_FORMAT.CONT_THOUSANDS_SEPS", + "sqlState" : "42601", + "messageParameters" : { + "format" : "'999G'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 54, + "fragment" : "to_number('123,000','999G')" + } ] +} + + +-- !query +SELECT '' AS to_number_16, to_number('123456','999G999') +-- !query analysis +Project [ AS to_number_16#x, to_number(123456, 999G999) AS to_number(123456, 999G999)#x] ++- OneRowRelation + + +-- !query +CREATE TABLE num_input_test (n1 decimal(38, 18)) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`num_input_test`, false + + +-- !query +INSERT INTO num_input_test VALUES (double(trim(' 123'))) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_input_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_input_test], Append, `spark_catalog`.`default`.`num_input_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_input_test), [n1] ++- Project [cast(col1#x as decimal(38,18)) AS n1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO num_input_test VALUES (double(trim(' 3245874 '))) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_input_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_input_test], Append, `spark_catalog`.`default`.`num_input_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_input_test), [n1] ++- Project [cast(col1#x as decimal(38,18)) AS n1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO num_input_test VALUES (double(trim(' -93853'))) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_input_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_input_test], Append, `spark_catalog`.`default`.`num_input_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_input_test), [n1] ++- Project [cast(col1#x as decimal(38,18)) AS n1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO num_input_test VALUES (555.50) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_input_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_input_test], Append, `spark_catalog`.`default`.`num_input_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_input_test), [n1] ++- Project [cast(col1#x as decimal(38,18)) AS n1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO num_input_test VALUES (-555.50) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_input_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_input_test], Append, `spark_catalog`.`default`.`num_input_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_input_test), [n1] ++- Project [cast(col1#x as decimal(38,18)) AS n1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT * FROM num_input_test +-- !query analysis +Project [n1#x] ++- SubqueryAlias spark_catalog.default.num_input_test + +- Relation spark_catalog.default.num_input_test[n1#x] parquet + + +-- !query +select cast(999999999999999999999 as decimal(38, 0))/1000000000000000000000 +-- !query analysis +Project [(cast(999999999999999999999 as decimal(38,0)) / 1000000000000000000000) AS (CAST(999999999999999999999 AS DECIMAL(38,0)) / 1000000000000000000000)#x] ++- OneRowRelation + + +-- !query +select div(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query analysis +Project [(cast(999999999999999999999 as decimal(38,0)) div 1000000000000000000000) AS (CAST(999999999999999999999 AS DECIMAL(38,0)) div 1000000000000000000000)#xL] ++- OneRowRelation + + +-- !query +select mod(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query analysis +Project [mod(cast(999999999999999999999 as decimal(38,0)), 1000000000000000000000) AS mod(CAST(999999999999999999999 AS DECIMAL(38,0)), 1000000000000000000000)#x] ++- OneRowRelation + + +-- !query +select div(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query analysis +Project [(cast(-9999999999999999999999 as decimal(38,0)) div 1000000000000000000000) AS (CAST(-9999999999999999999999 AS DECIMAL(38,0)) div 1000000000000000000000)#xL] ++- OneRowRelation + + +-- !query +select mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query analysis +Project [mod(cast(-9999999999999999999999 as decimal(38,0)), 1000000000000000000000) AS mod(CAST(-9999999999999999999999 AS DECIMAL(38,0)), 1000000000000000000000)#x] ++- OneRowRelation + + +-- !query +select div(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000)*1000000000000000000000 + mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query analysis +Project [((cast((cast(-9999999999999999999999 as decimal(38,0)) div 1000000000000000000000) as decimal(20,0)) * 1000000000000000000000) + mod(cast(-9999999999999999999999 as decimal(38,0)), 1000000000000000000000)) AS (((CAST(-9999999999999999999999 AS DECIMAL(38,0)) div 1000000000000000000000) * 1000000000000000000000) + mod(CAST(-9999999999999999999999 AS DECIMAL(38,0)), 1000000000000000000000))#x] ++- OneRowRelation + + +-- !query +select mod (70.0,70) +-- !query analysis +Project [mod(70.0, cast(70 as decimal(2,0))) AS mod(70.0, 70)#x] ++- OneRowRelation + + +-- !query +select div (70.0,70) +-- !query analysis +Project [(70.0 div cast(70 as decimal(2,0))) AS (70.0 div 70)#xL] ++- OneRowRelation + + +-- !query +select 70.0 / 70 +-- !query analysis +Project [(70.0 / cast(70 as decimal(2,0))) AS (70.0 / 70)#x] ++- OneRowRelation + + +-- !query +select 12345678901234567890 % 123 +-- !query analysis +Project [(12345678901234567890 % cast(123 as decimal(3,0))) AS (12345678901234567890 % 123)#x] ++- OneRowRelation + + +-- !query +select exp(0.0) +-- !query analysis +Project [EXP(cast(0.0 as double)) AS EXP(0.0)#x] ++- OneRowRelation + + +-- !query +select exp(1.0) +-- !query analysis +Project [EXP(cast(1.0 as double)) AS EXP(1.0)#x] ++- OneRowRelation + + +-- !query +select exp(32.999) +-- !query analysis +Project [EXP(cast(32.999 as double)) AS EXP(32.999)#x] ++- OneRowRelation + + +-- !query +select exp(-32.999) +-- !query analysis +Project [EXP(cast(-32.999 as double)) AS EXP(-32.999)#x] ++- OneRowRelation + + +-- !query +select exp(123.456) +-- !query analysis +Project [EXP(cast(123.456 as double)) AS EXP(123.456)#x] ++- OneRowRelation + + +-- !query +select exp(-123.456) +-- !query analysis +Project [EXP(cast(-123.456 as double)) AS EXP(-123.456)#x] ++- OneRowRelation + + +-- !query +select exp(1234.5678) +-- !query analysis +Project [EXP(cast(1234.5678 as double)) AS EXP(1234.5678)#x] ++- OneRowRelation + + +-- !query +select * from range(cast(0.0 as decimal(38, 18)), cast(4.0 as decimal(38, 18))) +-- !query analysis +Project [id#xL] ++- Range (0, 4, step=1, splits=None) + + +-- !query +select * from range(cast(0.1 as decimal(38, 18)), cast(4.0 as decimal(38, 18)), cast(1.3 as decimal(38, 18))) +-- !query analysis +Project [id#xL] ++- Range (0, 4, step=1, splits=None) + + +-- !query +select * from range(cast(4.0 as decimal(38, 18)), cast(-1.5 as decimal(38, 18)), cast(-2.2 as decimal(38, 18))) +-- !query analysis +Project [id#xL] ++- Range (4, -1, step=-2, splits=None) + + +-- !query +select ln(1.2345678e-28) +-- !query analysis +Project [ln(1.2345678E-28) AS ln(1.2345678E-28)#x] ++- OneRowRelation + + +-- !query +select ln(0.0456789) +-- !query analysis +Project [ln(cast(0.0456789 as double)) AS ln(0.0456789)#x] ++- OneRowRelation + + +-- !query +select ln(0.99949452) +-- !query analysis +Project [ln(cast(0.99949452 as double)) AS ln(0.99949452)#x] ++- OneRowRelation + + +-- !query +select ln(1.00049687395) +-- !query analysis +Project [ln(cast(1.00049687395 as double)) AS ln(1.00049687395)#x] ++- OneRowRelation + + +-- !query +select ln(1234.567890123456789) +-- !query analysis +Project [ln(cast(1234.567890123456789 as double)) AS ln(1234.567890123456789)#x] ++- OneRowRelation + + +-- !query +select ln(5.80397490724e5) +-- !query analysis +Project [ln(580397.490724) AS ln(580397.490724)#x] ++- OneRowRelation + + +-- !query +select ln(9.342536355e34) +-- !query analysis +Project [ln(9.342536355E34) AS ln(9.342536355E34)#x] ++- OneRowRelation + + +-- !query +select log(3.4634998359873254962349856073435545) +-- !query analysis +Project [LOG(E(), cast(3.4634998359873254962349856073435545 as double)) AS LOG(E(), 3.4634998359873254962349856073435545)#x] ++- OneRowRelation + + +-- !query +select log(9.999999999999999999) +-- !query analysis +Project [LOG(E(), cast(9.999999999999999999 as double)) AS LOG(E(), 9.999999999999999999)#x] ++- OneRowRelation + + +-- !query +select log(10.00000000000000000) +-- !query analysis +Project [LOG(E(), cast(10.00000000000000000 as double)) AS LOG(E(), 10.00000000000000000)#x] ++- OneRowRelation + + +-- !query +select log(10.00000000000000001) +-- !query analysis +Project [LOG(E(), cast(10.00000000000000001 as double)) AS LOG(E(), 10.00000000000000001)#x] ++- OneRowRelation + + +-- !query +select log(590489.45235237) +-- !query analysis +Project [LOG(E(), cast(590489.45235237 as double)) AS LOG(E(), 590489.45235237)#x] ++- OneRowRelation + + +-- !query +select log(0.99923, 4.58934e34) +-- !query analysis +Project [LOG(cast(0.99923 as double), 4.58934E34) AS LOG(0.99923, 4.58934E34)#x] ++- OneRowRelation + + +-- !query +select log(1.000016, 8.452010e18) +-- !query analysis +Project [LOG(cast(1.000016 as double), 8.45201E18) AS LOG(1.000016, 8.45201E18)#x] ++- OneRowRelation + + +-- !query +SELECT SUM(decimal(9999)) FROM range(1, 100001) +-- !query analysis +Aggregate [sum(cast(9999 as decimal(10,0))) AS sum(9999)#x] ++- Range (1, 100001, step=1, splits=None) + + +-- !query +SELECT SUM(decimal(-9999)) FROM range(1, 100001) +-- !query analysis +Aggregate [sum(cast(-9999 as decimal(10,0))) AS sum(-9999)#x] ++- Range (1, 100001, step=1, splits=None) + + +-- !query +DROP TABLE num_data +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`num_data`, false, false, false + + +-- !query +DROP TABLE num_exp_add +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`num_exp_add`, false, false, false + + +-- !query +DROP TABLE num_exp_sub +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`num_exp_sub`, false, false, false + + +-- !query +DROP TABLE num_exp_div +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`num_exp_div`, false, false, false + + +-- !query +DROP TABLE num_exp_mul +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`num_exp_mul`, false, false, false + + +-- !query +DROP TABLE num_exp_sqrt +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`num_exp_sqrt`, false, false, false + + +-- !query +DROP TABLE num_exp_ln +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`num_exp_ln`, false, false, false + + +-- !query +DROP TABLE num_exp_log10 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`num_exp_log10`, false, false, false + + +-- !query +DROP TABLE num_exp_power_10_ln +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`num_exp_power_10_ln`, false, false, false + + +-- !query +DROP TABLE num_result +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`num_result`, false, false, false + + +-- !query +DROP TABLE num_input_test +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`num_input_test`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select.sql.out new file mode 100644 index 0000000000000..3a0b4c036660a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select.sql.out @@ -0,0 +1,458 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create or replace temporary view onek2 as select * from onek +-- !query analysis +CreateViewCommand `onek2`, select * from onek, false, true, LocalTempView, true + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +create or replace temporary view INT8_TBL as select * from values + (cast(trim(' 123 ') as bigint), cast(trim(' 456') as bigint)), + (cast(trim('123 ') as bigint),cast('4567890123456789' as bigint)), + (cast('4567890123456789' as bigint),cast('123' as bigint)), + (cast(+4567890123456789 as bigint),cast('4567890123456789' as bigint)), + (cast('+4567890123456789' as bigint),cast('-4567890123456789' as bigint)) + as INT8_TBL(q1, q2) +-- !query analysis +CreateViewCommand `INT8_TBL`, select * from values + (cast(trim(' 123 ') as bigint), cast(trim(' 456') as bigint)), + (cast(trim('123 ') as bigint),cast('4567890123456789' as bigint)), + (cast('4567890123456789' as bigint),cast('123' as bigint)), + (cast(+4567890123456789 as bigint),cast('4567890123456789' as bigint)), + (cast('+4567890123456789' as bigint),cast('-4567890123456789' as bigint)) + as INT8_TBL(q1, q2), false, true, LocalTempView, true + +- Project [q1#xL, q2#xL] + +- SubqueryAlias INT8_TBL + +- LocalRelation [q1#xL, q2#xL] + + +-- !query +SELECT * FROM onek + WHERE onek.unique1 < 10 + ORDER BY onek.unique1 +-- !query analysis +Sort [unique1#x ASC NULLS FIRST], true ++- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- Filter (unique1#x < 10) + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC +-- !query analysis +Sort [unique1#x DESC NULLS LAST], true ++- Project [unique1#x, stringu1#x] + +- Filter (unique1#x < 20) + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 > 980 + ORDER BY stringu1 ASC +-- !query analysis +Sort [stringu1#x ASC NULLS FIRST], true ++- Project [unique1#x, stringu1#x] + +- Filter (unique1#x > 980) + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 ASC, unique1 DESC +-- !query analysis +Sort [string4#x ASC NULLS FIRST, unique1#x DESC NULLS LAST], true ++- Project [unique1#x, string4#x] + +- Filter (unique1#x > 980) + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 DESC, unique1 ASC +-- !query analysis +Sort [string4#x DESC NULLS LAST, unique1#x ASC NULLS FIRST], true ++- Project [unique1#x, string4#x] + +- Filter (unique1#x > 980) + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC, string4 ASC +-- !query analysis +Sort [unique1#x DESC NULLS LAST, string4#x ASC NULLS FIRST], true ++- Project [unique1#x, string4#x] + +- Filter (unique1#x < 20) + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 ASC, string4 DESC +-- !query analysis +Sort [unique1#x ASC NULLS FIRST, string4#x DESC NULLS LAST], true ++- Project [unique1#x, string4#x] + +- Filter (unique1#x < 20) + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT onek2.* FROM onek2 WHERE onek2.unique1 < 10 +-- !query analysis +Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] ++- Filter (unique1#x < 10) + +- SubqueryAlias onek2 + +- View (`onek2`, [unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x]) + +- Project [cast(unique1#x as int) AS unique1#x, cast(unique2#x as int) AS unique2#x, cast(two#x as int) AS two#x, cast(four#x as int) AS four#x, cast(ten#x as int) AS ten#x, cast(twenty#x as int) AS twenty#x, cast(hundred#x as int) AS hundred#x, cast(thousand#x as int) AS thousand#x, cast(twothousand#x as int) AS twothousand#x, cast(fivethous#x as int) AS fivethous#x, cast(tenthous#x as int) AS tenthous#x, cast(odd#x as int) AS odd#x, cast(even#x as int) AS even#x, cast(stringu1#x as string) AS stringu1#x, cast(stringu2#x as string) AS stringu2#x, cast(string4#x as string) AS string4#x] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 < 20 + ORDER BY unique1 DESC +-- !query analysis +Sort [unique1#x DESC NULLS LAST], true ++- Project [unique1#x, stringu1#x] + +- Filter (unique1#x < 20) + +- SubqueryAlias onek2 + +- View (`onek2`, [unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x]) + +- Project [cast(unique1#x as int) AS unique1#x, cast(unique2#x as int) AS unique2#x, cast(two#x as int) AS two#x, cast(four#x as int) AS four#x, cast(ten#x as int) AS ten#x, cast(twenty#x as int) AS twenty#x, cast(hundred#x as int) AS hundred#x, cast(thousand#x as int) AS thousand#x, cast(twothousand#x as int) AS twothousand#x, cast(fivethous#x as int) AS fivethous#x, cast(tenthous#x as int) AS tenthous#x, cast(odd#x as int) AS odd#x, cast(even#x as int) AS even#x, cast(stringu1#x as string) AS stringu1#x, cast(stringu2#x as string) AS stringu2#x, cast(string4#x as string) AS string4#x] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 > 980 +-- !query analysis +Project [unique1#x, stringu1#x] ++- Filter (unique1#x > 980) + +- SubqueryAlias onek2 + +- View (`onek2`, [unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x]) + +- Project [cast(unique1#x as int) AS unique1#x, cast(unique2#x as int) AS unique2#x, cast(two#x as int) AS two#x, cast(four#x as int) AS four#x, cast(ten#x as int) AS ten#x, cast(twenty#x as int) AS twenty#x, cast(hundred#x as int) AS hundred#x, cast(thousand#x as int) AS thousand#x, cast(twothousand#x as int) AS twothousand#x, cast(fivethous#x as int) AS fivethous#x, cast(tenthous#x as int) AS tenthous#x, cast(odd#x as int) AS odd#x, cast(even#x as int) AS even#x, cast(stringu1#x as string) AS stringu1#x, cast(stringu2#x as string) AS stringu2#x, cast(string4#x as string) AS string4#x] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +CREATE TABLE tmp USING parquet AS +SELECT two, stringu1, ten, string4 +FROM onek +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`tmp`, ErrorIfExists, [two, stringu1, ten, string4] + +- Project [two#x, stringu1#x, ten#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select foo.* from (select 1) as foo +-- !query analysis +Project [1#x] ++- SubqueryAlias foo + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +select foo.* from (select null) as foo +-- !query analysis +Project [NULL#x] ++- SubqueryAlias foo + +- Project [null AS NULL#x] + +- OneRowRelation + + +-- !query +select foo.* from (select 'xyzzy',1,null) as foo +-- !query analysis +Project [xyzzy#x, 1#x, NULL#x] ++- SubqueryAlias foo + +- Project [xyzzy AS xyzzy#x, 1 AS 1#x, null AS NULL#x] + +- OneRowRelation + + +-- !query +select * from onek, values(147, 'RFAAAA'), (931, 'VJAAAA') as v (i, j) + WHERE onek.unique1 = v.i and onek.stringu1 = v.j +-- !query analysis +Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, i#x, j#x] ++- Filter ((unique1#x = i#x) AND (stringu1#x = j#x)) + +- Join Inner + :- SubqueryAlias spark_catalog.default.onek + : +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias v + +- LocalRelation [i#x, j#x] + + +-- !query +VALUES (1,2), (3,4+4), (7,77.7) +-- !query analysis +LocalRelation [col1#x, col2#x] + + +-- !query +VALUES (1,2), (3,4+4), (7,77.7) +UNION ALL +SELECT 2+2, 57 +UNION ALL +TABLE int8_tbl +-- !query analysis +Union false, false +:- Project [cast(col1#x as bigint) AS col1#xL, cast(col2#x as decimal(21,1)) AS col2#x] +: +- Union false, false +: :- LocalRelation [col1#x, col2#x] +: +- Project [(2 + 2)#x, cast(57#x as decimal(11,1)) AS 57#x] +: +- Project [(2 + 2) AS (2 + 2)#x, 57 AS 57#x] +: +- OneRowRelation ++- Project [q1#xL, cast(q2#xL as decimal(21,1)) AS q2#x] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias INT8_TBL + +- LocalRelation [q1#xL, q2#xL] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW foo AS +SELECT * FROM (values(42),(3),(10),(7),(null),(null),(1)) as foo (f1) +-- !query analysis +CreateViewCommand `foo`, SELECT * FROM (values(42),(3),(10),(7),(null),(null),(1)) as foo (f1), false, true, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias foo + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT * FROM foo ORDER BY f1 +-- !query analysis +Sort [f1#x ASC NULLS FIRST], true ++- Project [f1#x] + +- SubqueryAlias foo + +- View (`foo`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias foo + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT * FROM foo ORDER BY f1 ASC +-- !query analysis +Sort [f1#x ASC NULLS FIRST], true ++- Project [f1#x] + +- SubqueryAlias foo + +- View (`foo`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias foo + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT * FROM foo ORDER BY f1 NULLS FIRST +-- !query analysis +Sort [f1#x ASC NULLS FIRST], true ++- Project [f1#x] + +- SubqueryAlias foo + +- View (`foo`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias foo + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT * FROM foo ORDER BY f1 DESC +-- !query analysis +Sort [f1#x DESC NULLS LAST], true ++- Project [f1#x] + +- SubqueryAlias foo + +- View (`foo`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias foo + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT * FROM foo ORDER BY f1 DESC NULLS LAST +-- !query analysis +Sort [f1#x DESC NULLS LAST], true ++- Project [f1#x] + +- SubqueryAlias foo + +- View (`foo`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias foo + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA' +-- !query analysis +Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] ++- Filter ((unique2#x = 11) AND (stringu1#x = ATAAAA)) + +- SubqueryAlias onek2 + +- View (`onek2`, [unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x]) + +- Project [cast(unique1#x as int) AS unique1#x, cast(unique2#x as int) AS unique2#x, cast(two#x as int) AS two#x, cast(four#x as int) AS four#x, cast(ten#x as int) AS ten#x, cast(twenty#x as int) AS twenty#x, cast(hundred#x as int) AS hundred#x, cast(thousand#x as int) AS thousand#x, cast(twothousand#x as int) AS twothousand#x, cast(fivethous#x as int) AS fivethous#x, cast(tenthous#x as int) AS tenthous#x, cast(odd#x as int) AS odd#x, cast(even#x as int) AS even#x, cast(stringu1#x as string) AS stringu1#x, cast(stringu2#x as string) AS stringu2#x, cast(string4#x as string) AS string4#x] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select unique2 from onek2 where unique2 = 11 and stringu1 = 'ATAAAA' +-- !query analysis +Project [unique2#x] ++- Filter ((unique2#x = 11) AND (stringu1#x = ATAAAA)) + +- SubqueryAlias onek2 + +- View (`onek2`, [unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x]) + +- Project [cast(unique1#x as int) AS unique1#x, cast(unique2#x as int) AS unique2#x, cast(two#x as int) AS two#x, cast(four#x as int) AS four#x, cast(ten#x as int) AS ten#x, cast(twenty#x as int) AS twenty#x, cast(hundred#x as int) AS hundred#x, cast(thousand#x as int) AS thousand#x, cast(twothousand#x as int) AS twothousand#x, cast(fivethous#x as int) AS fivethous#x, cast(tenthous#x as int) AS tenthous#x, cast(odd#x as int) AS odd#x, cast(even#x as int) AS even#x, cast(stringu1#x as string) AS stringu1#x, cast(stringu2#x as string) AS stringu2#x, cast(string4#x as string) AS string4#x] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select * from onek2 where unique2 = 11 and stringu1 < 'B' +-- !query analysis +Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] ++- Filter ((unique2#x = 11) AND (stringu1#x < B)) + +- SubqueryAlias onek2 + +- View (`onek2`, [unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x]) + +- Project [cast(unique1#x as int) AS unique1#x, cast(unique2#x as int) AS unique2#x, cast(two#x as int) AS two#x, cast(four#x as int) AS four#x, cast(ten#x as int) AS ten#x, cast(twenty#x as int) AS twenty#x, cast(hundred#x as int) AS hundred#x, cast(thousand#x as int) AS thousand#x, cast(twothousand#x as int) AS twothousand#x, cast(fivethous#x as int) AS fivethous#x, cast(tenthous#x as int) AS tenthous#x, cast(odd#x as int) AS odd#x, cast(even#x as int) AS even#x, cast(stringu1#x as string) AS stringu1#x, cast(stringu2#x as string) AS stringu2#x, cast(string4#x as string) AS string4#x] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' +-- !query analysis +Project [unique2#x] ++- Filter ((unique2#x = 11) AND (stringu1#x < B)) + +- SubqueryAlias onek2 + +- View (`onek2`, [unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x]) + +- Project [cast(unique1#x as int) AS unique1#x, cast(unique2#x as int) AS unique2#x, cast(two#x as int) AS two#x, cast(four#x as int) AS four#x, cast(ten#x as int) AS ten#x, cast(twenty#x as int) AS twenty#x, cast(hundred#x as int) AS hundred#x, cast(thousand#x as int) AS thousand#x, cast(twothousand#x as int) AS twothousand#x, cast(fivethous#x as int) AS fivethous#x, cast(tenthous#x as int) AS tenthous#x, cast(odd#x as int) AS odd#x, cast(even#x as int) AS even#x, cast(stringu1#x as string) AS stringu1#x, cast(stringu2#x as string) AS stringu2#x, cast(string4#x as string) AS string4#x] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select unique2 from onek2 where unique2 = 11 and stringu1 < 'C' +-- !query analysis +Project [unique2#x] ++- Filter ((unique2#x = 11) AND (stringu1#x < C)) + +- SubqueryAlias onek2 + +- View (`onek2`, [unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x]) + +- Project [cast(unique1#x as int) AS unique1#x, cast(unique2#x as int) AS unique2#x, cast(two#x as int) AS two#x, cast(four#x as int) AS four#x, cast(ten#x as int) AS ten#x, cast(twenty#x as int) AS twenty#x, cast(hundred#x as int) AS hundred#x, cast(thousand#x as int) AS thousand#x, cast(twothousand#x as int) AS twothousand#x, cast(fivethous#x as int) AS fivethous#x, cast(tenthous#x as int) AS tenthous#x, cast(odd#x as int) AS odd#x, cast(even#x as int) AS even#x, cast(stringu1#x as string) AS stringu1#x, cast(stringu2#x as string) AS stringu2#x, cast(string4#x as string) AS string4#x] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' +-- !query analysis +Project [unique2#x] ++- Filter ((unique2#x = 11) AND (stringu1#x < B)) + +- SubqueryAlias onek2 + +- View (`onek2`, [unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x]) + +- Project [cast(unique1#x as int) AS unique1#x, cast(unique2#x as int) AS unique2#x, cast(two#x as int) AS two#x, cast(four#x as int) AS four#x, cast(ten#x as int) AS ten#x, cast(twenty#x as int) AS twenty#x, cast(hundred#x as int) AS hundred#x, cast(thousand#x as int) AS thousand#x, cast(twothousand#x as int) AS twothousand#x, cast(fivethous#x as int) AS fivethous#x, cast(tenthous#x as int) AS tenthous#x, cast(odd#x as int) AS odd#x, cast(even#x as int) AS even#x, cast(stringu1#x as string) AS stringu1#x, cast(stringu2#x as string) AS stringu2#x, cast(string4#x as string) AS string4#x] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select unique1, unique2 from onek2 + where (unique2 = 11 or unique1 = 0) and stringu1 < 'B' +-- !query analysis +Project [unique1#x, unique2#x] ++- Filter (((unique2#x = 11) OR (unique1#x = 0)) AND (stringu1#x < B)) + +- SubqueryAlias onek2 + +- View (`onek2`, [unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x]) + +- Project [cast(unique1#x as int) AS unique1#x, cast(unique2#x as int) AS unique2#x, cast(two#x as int) AS two#x, cast(four#x as int) AS four#x, cast(ten#x as int) AS ten#x, cast(twenty#x as int) AS twenty#x, cast(hundred#x as int) AS hundred#x, cast(thousand#x as int) AS thousand#x, cast(twothousand#x as int) AS twothousand#x, cast(fivethous#x as int) AS fivethous#x, cast(tenthous#x as int) AS tenthous#x, cast(odd#x as int) AS odd#x, cast(even#x as int) AS even#x, cast(stringu1#x as string) AS stringu1#x, cast(stringu2#x as string) AS stringu2#x, cast(string4#x as string) AS string4#x] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select unique1, unique2 from onek2 + where (unique2 = 11 and stringu1 < 'B') or unique1 = 0 +-- !query analysis +Project [unique1#x, unique2#x] ++- Filter (((unique2#x = 11) AND (stringu1#x < B)) OR (unique1#x = 0)) + +- SubqueryAlias onek2 + +- View (`onek2`, [unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x]) + +- Project [cast(unique1#x as int) AS unique1#x, cast(unique2#x as int) AS unique2#x, cast(two#x as int) AS two#x, cast(four#x as int) AS four#x, cast(ten#x as int) AS ten#x, cast(twenty#x as int) AS twenty#x, cast(hundred#x as int) AS hundred#x, cast(thousand#x as int) AS thousand#x, cast(twothousand#x as int) AS twothousand#x, cast(fivethous#x as int) AS fivethous#x, cast(tenthous#x as int) AS tenthous#x, cast(odd#x as int) AS odd#x, cast(even#x as int) AS even#x, cast(stringu1#x as string) AS stringu1#x, cast(stringu2#x as string) AS stringu2#x, cast(string4#x as string) AS string4#x] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT 1 AS x ORDER BY x +-- !query analysis +Sort [x#x ASC NULLS FIRST], true ++- Project [1 AS x#x] + +- OneRowRelation + + +-- !query +select * from (values (2),(null),(1)) v(k) where k = k order by k +-- !query analysis +Sort [k#x ASC NULLS FIRST], true ++- Project [k#x] + +- Filter (k#x = k#x) + +- SubqueryAlias v + +- Project [col1#x AS k#x] + +- LocalRelation [col1#x] + + +-- !query +select * from (values (2),(null),(1)) v(k) where k = k +-- !query analysis +Project [k#x] ++- Filter (k#x = k#x) + +- SubqueryAlias v + +- Project [col1#x AS k#x] + +- LocalRelation [col1#x] + + +-- !query +drop table tmp +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tmp`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_distinct.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_distinct.sql.out new file mode 100644 index 0000000000000..9ede4682842d5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_distinct.sql.out @@ -0,0 +1,203 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW tmp AS +SELECT two, stringu1, ten, string4 +FROM onek +-- !query analysis +CreateViewCommand `tmp`, SELECT two, stringu1, ten, string4 +FROM onek, false, true, LocalTempView, true + +- Project [two#x, stringu1#x, ten#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT DISTINCT two FROM tmp ORDER BY 1 +-- !query analysis +Sort [two#x ASC NULLS FIRST], true ++- Distinct + +- Project [two#x] + +- SubqueryAlias tmp + +- View (`tmp`, [two#x,stringu1#x,ten#x,string4#x]) + +- Project [cast(two#x as int) AS two#x, cast(stringu1#x as string) AS stringu1#x, cast(ten#x as int) AS ten#x, cast(string4#x as string) AS string4#x] + +- Project [two#x, stringu1#x, ten#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT DISTINCT ten FROM tmp ORDER BY 1 +-- !query analysis +Sort [ten#x ASC NULLS FIRST], true ++- Distinct + +- Project [ten#x] + +- SubqueryAlias tmp + +- View (`tmp`, [two#x,stringu1#x,ten#x,string4#x]) + +- Project [cast(two#x as int) AS two#x, cast(stringu1#x as string) AS stringu1#x, cast(ten#x as int) AS ten#x, cast(string4#x as string) AS string4#x] + +- Project [two#x, stringu1#x, ten#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT DISTINCT string4 FROM tmp ORDER BY 1 +-- !query analysis +Sort [string4#x ASC NULLS FIRST], true ++- Distinct + +- Project [string4#x] + +- SubqueryAlias tmp + +- View (`tmp`, [two#x,stringu1#x,ten#x,string4#x]) + +- Project [cast(two#x as int) AS two#x, cast(stringu1#x as string) AS stringu1#x, cast(ten#x as int) AS ten#x, cast(string4#x as string) AS string4#x] + +- Project [two#x, stringu1#x, ten#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT DISTINCT two, string4, ten + FROM tmp + ORDER BY two ASC, string4 ASC, ten ASC +-- !query analysis +Sort [two#x ASC NULLS FIRST, string4#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true ++- Distinct + +- Project [two#x, string4#x, ten#x] + +- SubqueryAlias tmp + +- View (`tmp`, [two#x,stringu1#x,ten#x,string4#x]) + +- Project [cast(two#x as int) AS two#x, cast(stringu1#x as string) AS stringu1#x, cast(ten#x as int) AS ten#x, cast(string4#x as string) AS string4#x] + +- Project [two#x, stringu1#x, ten#x, string4#x] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT count(*) FROM + (SELECT DISTINCT two, four, two FROM tenk1) ss +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- SubqueryAlias ss + +- Distinct + +- Project [two#x, four#x, two#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW disttable AS SELECT * FROM + (VALUES (1), (2), (3), (NULL)) + AS v(f1) +-- !query analysis +CreateViewCommand `disttable`, SELECT * FROM + (VALUES (1), (2), (3), (NULL)) + AS v(f1), false, true, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT f1, f1 IS DISTINCT FROM 2 as `not 2` FROM disttable +-- !query analysis +Project [f1#x, NOT (f1#x <=> 2) AS not 2#x] ++- SubqueryAlias disttable + +- View (`disttable`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT f1, f1 IS DISTINCT FROM NULL as `not null` FROM disttable +-- !query analysis +Project [f1#x, NOT (f1#x <=> cast(null as int)) AS not null#x] ++- SubqueryAlias disttable + +- View (`disttable`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT f1, f1 IS DISTINCT FROM f1 as `false` FROM disttable +-- !query analysis +Project [f1#x, NOT (f1#x <=> f1#x) AS false#x] ++- SubqueryAlias disttable + +- View (`disttable`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT f1, f1 IS DISTINCT FROM f1+1 as `not null` FROM disttable +-- !query analysis +Project [f1#x, NOT (f1#x <=> (f1#x + 1)) AS not null#x] ++- SubqueryAlias disttable + +- View (`disttable`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT 1 IS DISTINCT FROM 2 as `yes` +-- !query analysis +Project [NOT (1 <=> 2) AS yes#x] ++- OneRowRelation + + +-- !query +SELECT 2 IS DISTINCT FROM 2 as `no` +-- !query analysis +Project [NOT (2 <=> 2) AS no#x] ++- OneRowRelation + + +-- !query +SELECT 2 IS DISTINCT FROM null as `yes` +-- !query analysis +Project [NOT (2 <=> cast(null as int)) AS yes#x] ++- OneRowRelation + + +-- !query +SELECT null IS DISTINCT FROM null as `no` +-- !query analysis +Project [NOT (null <=> null) AS no#x] ++- OneRowRelation + + +-- !query +SELECT 1 IS NOT DISTINCT FROM 2 as `no` +-- !query analysis +Project [(1 <=> 2) AS no#x] ++- OneRowRelation + + +-- !query +SELECT 2 IS NOT DISTINCT FROM 2 as `yes` +-- !query analysis +Project [(2 <=> 2) AS yes#x] ++- OneRowRelation + + +-- !query +SELECT 2 IS NOT DISTINCT FROM null as `no` +-- !query analysis +Project [(2 <=> cast(null as int)) AS no#x] ++- OneRowRelation + + +-- !query +SELECT null IS NOT DISTINCT FROM null as `yes` +-- !query analysis +Project [(null <=> null) AS yes#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_having.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_having.sql.out new file mode 100644 index 0000000000000..a841d56b34657 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_having.sql.out @@ -0,0 +1,211 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE test_having (a int, b int, c string, d string) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`test_having`, false + + +-- !query +INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (1, 2, 'AAAA', 'b') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (2, 2, 'AAAA', 'c') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (3, 3, 'BBBB', 'D') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (4, 3, 'BBBB', 'e') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (5, 3, 'bbbb', 'F') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (6, 4, 'cccc', 'g') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (7, 4, 'cccc', 'h') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (8, 4, 'CCCC', 'I') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (9, 4, 'CCCC', 'j') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +SELECT b, c FROM test_having + GROUP BY b, c HAVING count(*) = 1 ORDER BY b, c +-- !query analysis +Sort [b#x ASC NULLS FIRST, c#x ASC NULLS FIRST], true ++- Project [b#x, c#x] + +- Filter (count(1)#xL = cast(1 as bigint)) + +- Aggregate [b#x, c#x], [b#x, c#x, count(1) AS count(1)#xL] + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT b, c FROM test_having + GROUP BY b, c HAVING b = 3 ORDER BY b, c +-- !query analysis +Sort [b#x ASC NULLS FIRST, c#x ASC NULLS FIRST], true ++- Filter (b#x = 3) + +- Aggregate [b#x, c#x], [b#x, c#x] + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT c, max(a) FROM test_having + GROUP BY c HAVING count(*) > 2 OR min(a) = max(a) + ORDER BY c +-- !query analysis +Sort [c#x ASC NULLS FIRST], true ++- Project [c#x, max(a)#x] + +- Filter ((count(1)#xL > cast(2 as bigint)) OR (min(a#x)#x = max(a)#x)) + +- Aggregate [c#x], [c#x, max(a#x) AS max(a)#x, count(1) AS count(1)#xL, min(a#x) AS min(a#x)#x] + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT min(a), max(a) FROM test_having HAVING min(a) = max(a) +-- !query analysis +Filter (min(a)#x = max(a)#x) ++- Aggregate [min(a#x) AS min(a)#x, max(a#x) AS max(a)#x] + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT min(a), max(a) FROM test_having HAVING min(a) < max(a) +-- !query analysis +Filter (min(a)#x < max(a)#x) ++- Aggregate [min(a#x) AS min(a)#x, max(a#x) AS max(a)#x] + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT a FROM test_having HAVING min(a) < max(a) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 48, + "fragment" : "SELECT a FROM test_having HAVING min(a) < max(a)" + } ] +} + + +-- !query +SELECT 1 AS one FROM test_having HAVING a > 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`", + "proposal" : "`one`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 41, + "stopIndex" : 41, + "fragment" : "a" + } ] +} + + +-- !query +SELECT 1 AS one FROM test_having HAVING 1 > 2 +-- !query analysis +Filter (one#x > 2) ++- Aggregate [1 AS one#x] + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT 1 AS one FROM test_having HAVING 1 < 2 +-- !query analysis +Filter (one#x < 2) ++- Aggregate [1 AS one#x] + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT 1 AS one FROM test_having WHERE 1/a = 1 HAVING 1 < 2 +-- !query analysis +Filter (one#x < 2) ++- Aggregate [1 AS one#x] + +- Filter ((cast(1 as double) / cast(a#x as double)) = cast(1 as double)) + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +DROP TABLE test_having +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`test_having`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_implicit.sql.out new file mode 100644 index 0000000000000..417b433b5d945 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_implicit.sql.out @@ -0,0 +1,447 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE test_missing_target (a int, b int, c string, d string) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`test_missing_target`, false + + +-- !query +INSERT INTO test_missing_target VALUES (0, 1, 'XXXX', 'A') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (1, 2, 'ABAB', 'b') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (2, 2, 'ABAB', 'c') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (3, 3, 'BBBB', 'D') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (4, 3, 'BBBB', 'e') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (5, 3, 'bbbb', 'F') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (6, 4, 'cccc', 'g') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (7, 4, 'cccc', 'h') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (8, 4, 'CCCC', 'I') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (9, 4, 'CCCC', 'j') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +SELECT c, count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c +-- !query analysis +Sort [c#x ASC NULLS FIRST], true ++- Aggregate [c#x], [c#x, count(1) AS count(1)#xL] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c +-- !query analysis +Project [count(1)#xL] ++- Sort [c#x ASC NULLS FIRST], true + +- Aggregate [c#x], [count(1) AS count(1)#xL, c#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`b`", + "proposal" : "`count(1)`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 62, + "fragment" : "b" + } ] +} + + +-- !query +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b +-- !query analysis +Project [count(1)#xL] ++- Sort [b#x ASC NULLS FIRST], true + +- Aggregate [b#x], [count(1) AS count(1)#xL, b#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT test_missing_target.b, count(*) + FROM test_missing_target GROUP BY b ORDER BY b +-- !query analysis +Sort [b#x ASC NULLS FIRST], true ++- Aggregate [b#x], [b#x, count(1) AS count(1)#xL] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT c FROM test_missing_target ORDER BY a +-- !query analysis +Project [c#x] ++- Sort [a#x ASC NULLS FIRST], true + +- Project [c#x, a#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b desc +-- !query analysis +Project [count(1)#xL] ++- Sort [b#x DESC NULLS LAST], true + +- Aggregate [b#x], [count(1) AS count(1)#xL, b#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT count(*) FROM test_missing_target ORDER BY 1 desc +-- !query analysis +Sort [count(1)#xL DESC NULLS LAST], true ++- Aggregate [count(1) AS count(1)#xL] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT c, count(*) FROM test_missing_target GROUP BY 1 ORDER BY 1 +-- !query analysis +Sort [c#x ASC NULLS FIRST], true ++- Aggregate [c#x], [c#x, count(1) AS count(1)#xL] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT c, count(*) FROM test_missing_target GROUP BY 3 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "3", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 54, + "stopIndex" : 54, + "fragment" : "3" + } ] +} + + +-- !query +SELECT count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY b ORDER BY b +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`b`", + "referenceNames" : "[`x`.`b`, `y`.`b`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 94, + "stopIndex" : 94, + "fragment" : "b" + } ] +} + + +-- !query +SELECT a, a FROM test_missing_target + ORDER BY a +-- !query analysis +Sort [a#x ASC NULLS FIRST], true ++- Project [a#x, a#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT a/2, a/2 FROM test_missing_target + ORDER BY a/2 +-- !query analysis +Project [(a / 2)#x, (a / 2)#x] ++- Sort [(cast(a#x as double) / cast(2 as double)) ASC NULLS FIRST], true + +- Project [(cast(a#x as double) / cast(2 as double)) AS (a / 2)#x, (cast(a#x as double) / cast(2 as double)) AS (a / 2)#x, a#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT a/2, a/2 FROM test_missing_target + GROUP BY a/2 ORDER BY a/2 +-- !query analysis +Sort [(a / 2)#x ASC NULLS FIRST], true ++- Aggregate [(cast(a#x as double) / cast(2 as double))], [(cast(a#x as double) / cast(2 as double)) AS (a / 2)#x, (cast(a#x as double) / cast(2 as double)) AS (a / 2)#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT x.b, count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b ORDER BY x.b +-- !query analysis +Sort [b#x ASC NULLS FIRST], true ++- Aggregate [b#x], [b#x, count(1) AS count(1)#xL] + +- Filter (a#x = a#x) + +- Join Inner + :- SubqueryAlias x + : +- SubqueryAlias spark_catalog.default.test_missing_target + : +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + +- SubqueryAlias y + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b ORDER BY x.b +-- !query analysis +Project [count(1)#xL] ++- Sort [b#x ASC NULLS FIRST], true + +- Aggregate [b#x], [count(1) AS count(1)#xL, b#x] + +- Filter (a#x = a#x) + +- Join Inner + :- SubqueryAlias x + : +- SubqueryAlias spark_catalog.default.test_missing_target + : +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + +- SubqueryAlias y + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT a%2, count(b) FROM test_missing_target +GROUP BY test_missing_target.a%2 +ORDER BY test_missing_target.a%2 +-- !query analysis +Sort [(a % 2)#x ASC NULLS FIRST], true ++- Aggregate [(a#x % 2)], [(a#x % 2) AS (a % 2)#x, count(b#x) AS count(b)#xL] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT count(c) FROM test_missing_target +GROUP BY lower(test_missing_target.c) +ORDER BY lower(test_missing_target.c) +-- !query analysis +Project [count(c)#xL] ++- Sort [lower(c#x)#x ASC NULLS FIRST], true + +- Aggregate [lower(c#x)], [count(c#x) AS count(c)#xL, lower(c#x) AS lower(c#x)#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`b`", + "proposal" : "`count(a)`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 62, + "fragment" : "b" + } ] +} + + +-- !query +SELECT count(b) FROM test_missing_target GROUP BY b/2 ORDER BY b/2 +-- !query analysis +Project [count(b)#xL] ++- Sort [(cast(b#x as double) / cast(2 as double))#x ASC NULLS FIRST], true + +- Aggregate [(cast(b#x as double) / cast(2 as double))], [count(b#x) AS count(b)#xL, (cast(b#x as double) / cast(2 as double)) AS (cast(b#x as double) / cast(2 as double))#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT lower(test_missing_target.c), count(c) + FROM test_missing_target GROUP BY lower(c) ORDER BY lower(c) +-- !query analysis +Sort [lower(c)#x ASC NULLS FIRST], true ++- Aggregate [lower(c#x)], [lower(c#x) AS lower(c)#x, count(c#x) AS count(c)#xL] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT a FROM test_missing_target ORDER BY upper(d) +-- !query analysis +Project [a#x] ++- Sort [upper(d#x) ASC NULLS FIRST], true + +- Project [a#x, d#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT count(b) FROM test_missing_target + GROUP BY (b + 1) / 2 ORDER BY (b + 1) / 2 desc +-- !query analysis +Project [count(b)#xL] ++- Sort [(cast((b#x + 1) as double) / cast(2 as double))#x DESC NULLS LAST], true + +- Aggregate [(cast((b#x + 1) as double) / cast(2 as double))], [count(b#x) AS count(b)#xL, (cast((b#x + 1) as double) / cast(2 as double)) AS (cast((b#x + 1) as double) / cast(2 as double))#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT count(x.a) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY b/2 ORDER BY b/2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`b`", + "referenceNames" : "[`x`.`b`, `y`.`b`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 96, + "stopIndex" : 96, + "fragment" : "b" + } ] +} + + +-- !query +SELECT x.b/2, count(x.b) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b/2 ORDER BY x.b/2 +-- !query analysis +Sort [(b / 2)#x ASC NULLS FIRST], true ++- Aggregate [(cast(b#x as double) / cast(2 as double))], [(cast(b#x as double) / cast(2 as double)) AS (b / 2)#x, count(b#x) AS count(b)#xL] + +- Filter (a#x = a#x) + +- Join Inner + :- SubqueryAlias x + : +- SubqueryAlias spark_catalog.default.test_missing_target + : +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + +- SubqueryAlias y + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT count(b) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b/2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`b`", + "referenceNames" : "[`x`.`b`, `y`.`b`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 14, + "stopIndex" : 14, + "fragment" : "b" + } ] +} + + +-- !query +DROP TABLE test_missing_target +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`test_missing_target`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/strings.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/strings.sql.out new file mode 100644 index 0000000000000..ab078642e96ec --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/strings.sql.out @@ -0,0 +1,877 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT 'first line' +' - next line' + ' - third line' + AS `Three lines to one` +-- !query analysis +Project [first line - next line - third line AS Three lines to one#x] ++- OneRowRelation + + +-- !query +SELECT 'first line' +' - next line' /* this comment is not allowed here */ +' - third line' + AS `Illegal comment within continuation` +-- !query analysis +Project [first line - next line - third line AS Illegal comment within continuation#x] ++- OneRowRelation + + +-- !query +SELECT binary('\\xDeAdBeEf') +-- !query analysis +Project [cast(\xDeAdBeEf as binary) AS \xDeAdBeEf#x] ++- OneRowRelation + + +-- !query +SELECT binary('\\x De Ad Be Ef ') +-- !query analysis +Project [cast(\x De Ad Be Ef as binary) AS \x De Ad Be Ef #x] ++- OneRowRelation + + +-- !query +SELECT binary('\\xDe00BeEf') +-- !query analysis +Project [cast(\xDe00BeEf as binary) AS \xDe00BeEf#x] ++- OneRowRelation + + +-- !query +SELECT binary('DeAdBeEf') +-- !query analysis +Project [cast(DeAdBeEf as binary) AS DeAdBeEf#x] ++- OneRowRelation + + +-- !query +SELECT binary('De\\000dBeEf') +-- !query analysis +Project [cast(De\000dBeEf as binary) AS De\000dBeEf#x] ++- OneRowRelation + + +-- !query +SELECT binary('De\\123dBeEf') +-- !query analysis +Project [cast(De\123dBeEf as binary) AS De\123dBeEf#x] ++- OneRowRelation + + +-- !query +SELECT TRIM(BOTH FROM ' bunch o blanks ') = 'bunch o blanks' AS `bunch o blanks` +-- !query analysis +Project [(trim( bunch o blanks , None) = bunch o blanks) AS bunch o blanks#x] ++- OneRowRelation + + +-- !query +SELECT TRIM(LEADING FROM ' bunch o blanks ') = 'bunch o blanks ' AS `bunch o blanks ` +-- !query analysis +Project [(ltrim( bunch o blanks , None) = bunch o blanks ) AS bunch o blanks #x] ++- OneRowRelation + + +-- !query +SELECT TRIM(TRAILING FROM ' bunch o blanks ') = ' bunch o blanks' AS ` bunch o blanks` +-- !query analysis +Project [(rtrim( bunch o blanks , None) = bunch o blanks) AS bunch o blanks#x] ++- OneRowRelation + + +-- !query +SELECT TRIM(BOTH 'x' FROM 'xxxxxsome Xsxxxxx') = 'some Xs' AS `some Xs` +-- !query analysis +Project [(trim(xxxxxsome Xsxxxxx, Some(x)) = some Xs) AS some Xs#x] ++- OneRowRelation + + +-- !query +SELECT SUBSTRING('1234567890' FROM 3) = '34567890' AS `34567890` +-- !query analysis +Project [(substring(1234567890, 3, 2147483647) = 34567890) AS 34567890#x] ++- OneRowRelation + + +-- !query +SELECT SUBSTRING('1234567890' FROM 4 FOR 3) = '456' AS `456` +-- !query analysis +Project [(substring(1234567890, 4, 3) = 456) AS 456#x] ++- OneRowRelation + + +-- !query +SELECT POSITION('4' IN '1234567890') = '4' AS `4` +-- !query analysis +Project [(cast(locate(4, 1234567890, 1) as bigint) = cast(4 as bigint)) AS 4#x] ++- OneRowRelation + + +-- !query +SELECT POSITION('5' IN '1234567890') = '5' AS `5` +-- !query analysis +Project [(cast(locate(5, 1234567890, 1) as bigint) = cast(5 as bigint)) AS 5#x] ++- OneRowRelation + + +-- !query +SELECT OVERLAY('abcdef' PLACING '45' FROM 4) AS `abc45f` +-- !query analysis +Project [overlay(abcdef, 45, 4, -1) AS abc45f#x] ++- OneRowRelation + + +-- !query +SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5) AS `yabadaba` +-- !query analysis +Project [overlay(yabadoo, daba, 5, -1) AS yabadaba#x] ++- OneRowRelation + + +-- !query +SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5 FOR 0) AS `yabadabadoo` +-- !query analysis +Project [overlay(yabadoo, daba, 5, 0) AS yabadabadoo#x] ++- OneRowRelation + + +-- !query +SELECT OVERLAY('babosa' PLACING 'ubb' FROM 2 FOR 4) AS `bubba` +-- !query analysis +Project [overlay(babosa, ubb, 2, 4) AS bubba#x] ++- OneRowRelation + + +-- !query +SELECT 'hawkeye' LIKE 'h%' AS `true` +-- !query analysis +Project [hawkeye LIKE h% AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'hawkeye' NOT LIKE 'h%' AS `false` +-- !query analysis +Project [NOT hawkeye LIKE h% AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'hawkeye' LIKE 'H%' AS `false` +-- !query analysis +Project [hawkeye LIKE H% AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'hawkeye' NOT LIKE 'H%' AS `true` +-- !query analysis +Project [NOT hawkeye LIKE H% AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'hawkeye' LIKE 'indio%' AS `false` +-- !query analysis +Project [hawkeye LIKE indio% AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'hawkeye' NOT LIKE 'indio%' AS `true` +-- !query analysis +Project [NOT hawkeye LIKE indio% AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'hawkeye' LIKE 'h%eye' AS `true` +-- !query analysis +Project [hawkeye LIKE h%eye AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'hawkeye' NOT LIKE 'h%eye' AS `false` +-- !query analysis +Project [NOT hawkeye LIKE h%eye AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'indio' LIKE '_ndio' AS `true` +-- !query analysis +Project [indio LIKE _ndio AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'indio' NOT LIKE '_ndio' AS `false` +-- !query analysis +Project [NOT indio LIKE _ndio AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'indio' LIKE 'in__o' AS `true` +-- !query analysis +Project [indio LIKE in__o AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'indio' NOT LIKE 'in__o' AS `false` +-- !query analysis +Project [NOT indio LIKE in__o AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'indio' LIKE 'in_o' AS `false` +-- !query analysis +Project [indio LIKE in_o AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'indio' NOT LIKE 'in_o' AS `true` +-- !query analysis +Project [NOT indio LIKE in_o AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'hawkeye' LIKE 'h%' ESCAPE '#' AS `true` +-- !query analysis +Project [hawkeye LIKE h% ESCAPE '#' AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'hawkeye' NOT LIKE 'h%' ESCAPE '#' AS `false` +-- !query analysis +Project [NOT hawkeye LIKE h% ESCAPE '#' AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'indio' LIKE 'ind_o' ESCAPE '$' AS `true` +-- !query analysis +Project [indio LIKE ind_o ESCAPE '$' AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'indio' NOT LIKE 'ind_o' ESCAPE '$' AS `false` +-- !query analysis +Project [NOT indio LIKE ind_o ESCAPE '$' AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'h%' LIKE 'h#%' ESCAPE '#' AS `true` +-- !query analysis +Project [h% LIKE h#% ESCAPE '#' AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'h%' NOT LIKE 'h#%' ESCAPE '#' AS `false` +-- !query analysis +Project [NOT h% LIKE h#% ESCAPE '#' AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'h%wkeye' LIKE 'h#%' ESCAPE '#' AS `false` +-- !query analysis +Project [h%wkeye LIKE h#% ESCAPE '#' AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'h%wkeye' NOT LIKE 'h#%' ESCAPE '#' AS `true` +-- !query analysis +Project [NOT h%wkeye LIKE h#% ESCAPE '#' AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'h%wkeye' LIKE 'h#%%' ESCAPE '#' AS `true` +-- !query analysis +Project [h%wkeye LIKE h#%% ESCAPE '#' AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'h%wkeye' NOT LIKE 'h#%%' ESCAPE '#' AS `false` +-- !query analysis +Project [NOT h%wkeye LIKE h#%% ESCAPE '#' AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'h%awkeye' LIKE 'h#%a%k%e' ESCAPE '#' AS `true` +-- !query analysis +Project [h%awkeye LIKE h#%a%k%e ESCAPE '#' AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'h%awkeye' NOT LIKE 'h#%a%k%e' ESCAPE '#' AS `false` +-- !query analysis +Project [NOT h%awkeye LIKE h#%a%k%e ESCAPE '#' AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'indio' LIKE '_ndio' ESCAPE '$' AS `true` +-- !query analysis +Project [indio LIKE _ndio ESCAPE '$' AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'indio' NOT LIKE '_ndio' ESCAPE '$' AS `false` +-- !query analysis +Project [NOT indio LIKE _ndio ESCAPE '$' AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'i_dio' LIKE 'i$_d_o' ESCAPE '$' AS `true` +-- !query analysis +Project [i_dio LIKE i$_d_o ESCAPE '$' AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'i_dio' NOT LIKE 'i$_d_o' ESCAPE '$' AS `false` +-- !query analysis +Project [NOT i_dio LIKE i$_d_o ESCAPE '$' AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'i_dio' LIKE 'i$_nd_o' ESCAPE '$' AS `false` +-- !query analysis +Project [i_dio LIKE i$_nd_o ESCAPE '$' AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'i_dio' NOT LIKE 'i$_nd_o' ESCAPE '$' AS `true` +-- !query analysis +Project [NOT i_dio LIKE i$_nd_o ESCAPE '$' AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'i_dio' LIKE 'i$_d%o' ESCAPE '$' AS `true` +-- !query analysis +Project [i_dio LIKE i$_d%o ESCAPE '$' AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'i_dio' NOT LIKE 'i$_d%o' ESCAPE '$' AS `false` +-- !query analysis +Project [NOT i_dio LIKE i$_d%o ESCAPE '$' AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'maca' LIKE 'm%aca' ESCAPE '%' AS `true` +-- !query analysis +Project [maca LIKE m%aca ESCAPE '%' AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'maca' NOT LIKE 'm%aca' ESCAPE '%' AS `false` +-- !query analysis +Project [NOT maca LIKE m%aca ESCAPE '%' AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'ma%a' LIKE 'm%a%%a' ESCAPE '%' AS `true` +-- !query analysis +Project [ma%a LIKE m%a%%a ESCAPE '%' AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'ma%a' NOT LIKE 'm%a%%a' ESCAPE '%' AS `false` +-- !query analysis +Project [NOT ma%a LIKE m%a%%a ESCAPE '%' AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'bear' LIKE 'b_ear' ESCAPE '_' AS `true` +-- !query analysis +Project [bear LIKE b_ear ESCAPE '_' AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'bear' NOT LIKE 'b_ear' ESCAPE '_' AS `false` +-- !query analysis +Project [NOT bear LIKE b_ear ESCAPE '_' AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'be_r' LIKE 'b_e__r' ESCAPE '_' AS `true` +-- !query analysis +Project [be_r LIKE b_e__r ESCAPE '_' AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'be_r' NOT LIKE 'b_e__r' ESCAPE '_' AS `false` +-- !query analysis +Project [NOT be_r LIKE b_e__r ESCAPE '_' AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'be_r' LIKE '__e__r' ESCAPE '_' AS `false` +-- !query analysis +Project [be_r LIKE __e__r ESCAPE '_' AS false#x] ++- OneRowRelation + + +-- !query +SELECT 'be_r' NOT LIKE '__e__r' ESCAPE '_' AS `true` +-- !query analysis +Project [NOT be_r LIKE __e__r ESCAPE '_' AS true#x] ++- OneRowRelation + + +-- !query +SELECT 'foo' LIKE '_%' as t, 'f' LIKE '_%' as t, '' LIKE '_%' as f +-- !query analysis +Project [foo LIKE _% AS t#x, f LIKE _% AS t#x, LIKE _% AS f#x] ++- OneRowRelation + + +-- !query +SELECT 'foo' LIKE '%_' as t, 'f' LIKE '%_' as t, '' LIKE '%_' as f +-- !query analysis +Project [foo LIKE %_ AS t#x, f LIKE %_ AS t#x, LIKE %_ AS f#x] ++- OneRowRelation + + +-- !query +SELECT 'foo' LIKE '__%' as t, 'foo' LIKE '___%' as t, 'foo' LIKE '____%' as f +-- !query analysis +Project [foo LIKE __% AS t#x, foo LIKE ___% AS t#x, foo LIKE ____% AS f#x] ++- OneRowRelation + + +-- !query +SELECT 'foo' LIKE '%__' as t, 'foo' LIKE '%___' as t, 'foo' LIKE '%____' as f +-- !query analysis +Project [foo LIKE %__ AS t#x, foo LIKE %___ AS t#x, foo LIKE %____ AS f#x] ++- OneRowRelation + + +-- !query +SELECT 'jack' LIKE '%____%' AS t +-- !query analysis +Project [jack LIKE %____% AS t#x] ++- OneRowRelation + + +-- !query +SELECT 'unknown' || ' and unknown' AS `Concat unknown types` +-- !query analysis +Project [concat(unknown, and unknown) AS Concat unknown types#x] ++- OneRowRelation + + +-- !query +SELECT string('text') || ' and unknown' AS `Concat text to unknown type` +-- !query analysis +Project [concat(cast(text as string), and unknown) AS Concat text to unknown type#x] ++- OneRowRelation + + +-- !query +CREATE TABLE toasttest(f1 string) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`toasttest`, false + + +-- !query +insert into toasttest values(repeat('1234567890',10000)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/toasttest, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/toasttest], Append, `spark_catalog`.`default`.`toasttest`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/toasttest), [f1] ++- Project [cast(col1#x as string) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +insert into toasttest values(repeat('1234567890',10000)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/toasttest, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/toasttest], Append, `spark_catalog`.`default`.`toasttest`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/toasttest), [f1] ++- Project [cast(col1#x as string) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +insert into toasttest values(repeat('1234567890',10000)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/toasttest, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/toasttest], Append, `spark_catalog`.`default`.`toasttest`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/toasttest), [f1] ++- Project [cast(col1#x as string) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +insert into toasttest values(repeat('1234567890',10000)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/toasttest, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/toasttest], Append, `spark_catalog`.`default`.`toasttest`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/toasttest), [f1] ++- Project [cast(col1#x as string) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT substr(f1, 99995) from toasttest +-- !query analysis +Project [substr(f1#x, 99995, 2147483647) AS substr(f1, 99995, 2147483647)#x] ++- SubqueryAlias spark_catalog.default.toasttest + +- Relation spark_catalog.default.toasttest[f1#x] parquet + + +-- !query +SELECT substr(f1, 99995, 10) from toasttest +-- !query analysis +Project [substr(f1#x, 99995, 10) AS substr(f1, 99995, 10)#x] ++- SubqueryAlias spark_catalog.default.toasttest + +- Relation spark_catalog.default.toasttest[f1#x] parquet + + +-- !query +SELECT length('abcdef') AS `length_6` +-- !query analysis +Project [length(abcdef) AS length_6#x] ++- OneRowRelation + + +-- !query +SELECT position('cd', 'abcdef') AS `pos_3` +-- !query analysis +Project [position(cd, abcdef, 1) AS pos_3#x] ++- OneRowRelation + + +-- !query +SELECT position('xy', 'abcdef') AS `pos_0` +-- !query analysis +Project [position(xy, abcdef, 1) AS pos_0#x] ++- OneRowRelation + + +-- !query +SELECT replace('abcdef', 'de', '45') AS `abc45f` +-- !query analysis +Project [replace(abcdef, de, 45) AS abc45f#x] ++- OneRowRelation + + +-- !query +SELECT replace('yabadabadoo', 'ba', '123') AS `ya123da123doo` +-- !query analysis +Project [replace(yabadabadoo, ba, 123) AS ya123da123doo#x] ++- OneRowRelation + + +-- !query +SELECT replace('yabadoo', 'bad', '') AS `yaoo` +-- !query analysis +Project [replace(yabadoo, bad, ) AS yaoo#x] ++- OneRowRelation + + +-- !query +select hex(256*256*256 - 1) AS `ffffff` +-- !query analysis +Project [hex(cast((((256 * 256) * 256) - 1) as bigint)) AS ffffff#x] ++- OneRowRelation + + +-- !query +select hex(bigint(bigint(bigint(bigint(256)*256)*256)*256) - 1) AS `ffffffff` +-- !query analysis +Project [hex((cast((cast((cast((cast(256 as bigint) * cast(256 as bigint)) as bigint) * cast(256 as bigint)) as bigint) * cast(256 as bigint)) as bigint) - cast(1 as bigint))) AS ffffffff#x] ++- OneRowRelation + + +-- !query +select md5('') = 'd41d8cd98f00b204e9800998ecf8427e' AS `TRUE` +-- !query analysis +Project [(md5(cast( as binary)) = d41d8cd98f00b204e9800998ecf8427e) AS TRUE#x] ++- OneRowRelation + + +-- !query +select md5('a') = '0cc175b9c0f1b6a831c399e269772661' AS `TRUE` +-- !query analysis +Project [(md5(cast(a as binary)) = 0cc175b9c0f1b6a831c399e269772661) AS TRUE#x] ++- OneRowRelation + + +-- !query +select md5('abc') = '900150983cd24fb0d6963f7d28e17f72' AS `TRUE` +-- !query analysis +Project [(md5(cast(abc as binary)) = 900150983cd24fb0d6963f7d28e17f72) AS TRUE#x] ++- OneRowRelation + + +-- !query +select md5('message digest') = 'f96b697d7cb7938d525a2f31aaf161d0' AS `TRUE` +-- !query analysis +Project [(md5(cast(message digest as binary)) = f96b697d7cb7938d525a2f31aaf161d0) AS TRUE#x] ++- OneRowRelation + + +-- !query +select md5('abcdefghijklmnopqrstuvwxyz') = 'c3fcd3d76192e4007dfb496cca67e13b' AS `TRUE` +-- !query analysis +Project [(md5(cast(abcdefghijklmnopqrstuvwxyz as binary)) = c3fcd3d76192e4007dfb496cca67e13b) AS TRUE#x] ++- OneRowRelation + + +-- !query +select md5('ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789') = 'd174ab98d277d9f5a5611c2c9f419d9f' AS `TRUE` +-- !query analysis +Project [(md5(cast(ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789 as binary)) = d174ab98d277d9f5a5611c2c9f419d9f) AS TRUE#x] ++- OneRowRelation + + +-- !query +select md5('12345678901234567890123456789012345678901234567890123456789012345678901234567890') = '57edf4a22be3c955ac49da2e2107b67a' AS `TRUE` +-- !query analysis +Project [(md5(cast(12345678901234567890123456789012345678901234567890123456789012345678901234567890 as binary)) = 57edf4a22be3c955ac49da2e2107b67a) AS TRUE#x] ++- OneRowRelation + + +-- !query +select md5(binary('')) = 'd41d8cd98f00b204e9800998ecf8427e' AS `TRUE` +-- !query analysis +Project [(md5(cast( as binary)) = d41d8cd98f00b204e9800998ecf8427e) AS TRUE#x] ++- OneRowRelation + + +-- !query +select md5(binary('a')) = '0cc175b9c0f1b6a831c399e269772661' AS `TRUE` +-- !query analysis +Project [(md5(cast(a as binary)) = 0cc175b9c0f1b6a831c399e269772661) AS TRUE#x] ++- OneRowRelation + + +-- !query +select md5(binary('abc')) = '900150983cd24fb0d6963f7d28e17f72' AS `TRUE` +-- !query analysis +Project [(md5(cast(abc as binary)) = 900150983cd24fb0d6963f7d28e17f72) AS TRUE#x] ++- OneRowRelation + + +-- !query +select md5(binary('message digest')) = 'f96b697d7cb7938d525a2f31aaf161d0' AS `TRUE` +-- !query analysis +Project [(md5(cast(message digest as binary)) = f96b697d7cb7938d525a2f31aaf161d0) AS TRUE#x] ++- OneRowRelation + + +-- !query +select md5(binary('abcdefghijklmnopqrstuvwxyz')) = 'c3fcd3d76192e4007dfb496cca67e13b' AS `TRUE` +-- !query analysis +Project [(md5(cast(abcdefghijklmnopqrstuvwxyz as binary)) = c3fcd3d76192e4007dfb496cca67e13b) AS TRUE#x] ++- OneRowRelation + + +-- !query +select md5(binary('ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789')) = 'd174ab98d277d9f5a5611c2c9f419d9f' AS `TRUE` +-- !query analysis +Project [(md5(cast(ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789 as binary)) = d174ab98d277d9f5a5611c2c9f419d9f) AS TRUE#x] ++- OneRowRelation + + +-- !query +select md5(binary('12345678901234567890123456789012345678901234567890123456789012345678901234567890')) = '57edf4a22be3c955ac49da2e2107b67a' AS `TRUE` +-- !query analysis +Project [(md5(cast(12345678901234567890123456789012345678901234567890123456789012345678901234567890 as binary)) = 57edf4a22be3c955ac49da2e2107b67a) AS TRUE#x] ++- OneRowRelation + + +-- !query +SELECT initcap('hi THOMAS') +-- !query analysis +Project [initcap(hi THOMAS) AS initcap(hi THOMAS)#x] ++- OneRowRelation + + +-- !query +SELECT lpad('hi', 5, 'xy') +-- !query analysis +Project [lpad(hi, 5, xy) AS lpad(hi, 5, xy)#x] ++- OneRowRelation + + +-- !query +SELECT lpad('hi', 5) +-- !query analysis +Project [lpad(hi, 5, ) AS lpad(hi, 5, )#x] ++- OneRowRelation + + +-- !query +SELECT lpad('hi', -5, 'xy') +-- !query analysis +Project [lpad(hi, -5, xy) AS lpad(hi, -5, xy)#x] ++- OneRowRelation + + +-- !query +SELECT lpad('hello', 2) +-- !query analysis +Project [lpad(hello, 2, ) AS lpad(hello, 2, )#x] ++- OneRowRelation + + +-- !query +SELECT lpad('hi', 5, '') +-- !query analysis +Project [lpad(hi, 5, ) AS lpad(hi, 5, )#x] ++- OneRowRelation + + +-- !query +SELECT rpad('hi', 5, 'xy') +-- !query analysis +Project [rpad(hi, 5, xy) AS rpad(hi, 5, xy)#x] ++- OneRowRelation + + +-- !query +SELECT rpad('hi', 5) +-- !query analysis +Project [rpad(hi, 5, ) AS rpad(hi, 5, )#x] ++- OneRowRelation + + +-- !query +SELECT rpad('hi', -5, 'xy') +-- !query analysis +Project [rpad(hi, -5, xy) AS rpad(hi, -5, xy)#x] ++- OneRowRelation + + +-- !query +SELECT rpad('hello', 2) +-- !query analysis +Project [rpad(hello, 2, ) AS rpad(hello, 2, )#x] ++- OneRowRelation + + +-- !query +SELECT rpad('hi', 5, '') +-- !query analysis +Project [rpad(hi, 5, ) AS rpad(hi, 5, )#x] ++- OneRowRelation + + +-- !query +SELECT translate('', '14', 'ax') +-- !query analysis +Project [translate(, 14, ax) AS translate(, 14, ax)#x] ++- OneRowRelation + + +-- !query +SELECT translate('12345', '14', 'ax') +-- !query analysis +Project [translate(12345, 14, ax) AS translate(12345, 14, ax)#x] ++- OneRowRelation + + +-- !query +SELECT ascii('x') +-- !query analysis +Project [ascii(x) AS ascii(x)#x] ++- OneRowRelation + + +-- !query +SELECT ascii('') +-- !query analysis +Project [ascii() AS ascii()#x] ++- OneRowRelation + + +-- !query +SELECT chr(65) +-- !query analysis +Project [chr(cast(65 as bigint)) AS chr(65)#x] ++- OneRowRelation + + +-- !query +SELECT chr(0) +-- !query analysis +Project [chr(cast(0 as bigint)) AS chr(0)#x] ++- OneRowRelation + + +-- !query +SELECT repeat('Pg', 4) +-- !query analysis +Project [repeat(Pg, 4) AS repeat(Pg, 4)#x] ++- OneRowRelation + + +-- !query +SELECT repeat('Pg', -4) +-- !query analysis +Project [repeat(Pg, -4) AS repeat(Pg, -4)#x] ++- OneRowRelation + + +-- !query +SELECT trim(binary('\\000') from binary('\\000Tom\\000')) +-- !query analysis +Project [trim(cast(cast(\000Tom\000 as binary) as string), Some(cast(cast(\000 as binary) as string))) AS TRIM(BOTH \000 FROM \000Tom\000)#x] ++- OneRowRelation + + +-- !query +SELECT btrim(binary('\\000trim\\000'), binary('\\000')) +-- !query analysis +Project [btrim(cast(\000trim\000 as binary), cast(\000 as binary)) AS btrim(\000trim\000, \000)#x] ++- OneRowRelation + + +-- !query +SELECT btrim(binary(''), binary('\\000')) +-- !query analysis +Project [btrim(cast( as binary), cast(\000 as binary)) AS btrim(, \000)#x] ++- OneRowRelation + + +-- !query +SELECT btrim(binary('\\000trim\\000'), binary('')) +-- !query analysis +Project [btrim(cast(\000trim\000 as binary), cast( as binary)) AS btrim(\000trim\000, )#x] ++- OneRowRelation + + +-- !query +DROP TABLE toasttest +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`toasttest`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/text.sql.out new file mode 100644 index 0000000000000..16031e1d0ed8a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/text.sql.out @@ -0,0 +1,317 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT string('this is a text string') = string('this is a text string') AS true +-- !query analysis +Project [(cast(this is a text string as string) = cast(this is a text string as string)) AS true#x] ++- OneRowRelation + + +-- !query +SELECT string('this is a text string') = string('this is a text strin') AS `false` +-- !query analysis +Project [(cast(this is a text string as string) = cast(this is a text strin as string)) AS false#x] ++- OneRowRelation + + +-- !query +CREATE TABLE TEXT_TBL (f1 string) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`TEXT_TBL`, false + + +-- !query +INSERT INTO TEXT_TBL VALUES ('doh!') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/text_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/text_tbl], Append, `spark_catalog`.`default`.`text_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/text_tbl), [f1] ++- Project [cast(col1#x as string) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO TEXT_TBL VALUES ('hi de ho neighbor') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/text_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/text_tbl], Append, `spark_catalog`.`default`.`text_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/text_tbl), [f1] ++- Project [cast(col1#x as string) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT '' AS two, * FROM TEXT_TBL +-- !query analysis +Project [ AS two#x, f1#x] ++- SubqueryAlias spark_catalog.default.text_tbl + +- Relation spark_catalog.default.text_tbl[f1#x] parquet + + +-- !query +select length(42) +-- !query analysis +Project [length(cast(42 as string)) AS length(42)#x] ++- OneRowRelation + + +-- !query +select string('four: ') || 2+2 +-- !query analysis +Project [(cast(concat(cast(four: as string), cast(2 as string)) as bigint) + cast(2 as bigint)) AS (concat(four: , 2) + 2)#xL] ++- OneRowRelation + + +-- !query +select 'four: ' || 2+2 +-- !query analysis +Project [(cast(concat(four: , cast(2 as string)) as bigint) + cast(2 as bigint)) AS (concat(four: , 2) + 2)#xL] ++- OneRowRelation + + +-- !query +select 3 || 4.0 +-- !query analysis +Project [concat(cast(3 as string), cast(4.0 as string)) AS concat(3, 4.0)#x] ++- OneRowRelation + + +-- !query +/* + * various string functions + */ +select concat('one') +-- !query analysis +Project [concat(one) AS concat(one)#x] ++- OneRowRelation + + +-- !query +select concat(1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')) +-- !query analysis +Project [concat(cast(1 as string), cast(2 as string), cast(3 as string), hello, cast(true as string), cast(false as string), cast(to_date(20100309, Some(yyyyMMdd), Some(America/Los_Angeles)) as string)) AS concat(1, 2, 3, hello, true, false, to_date(20100309, yyyyMMdd))#x] ++- OneRowRelation + + +-- !query +select concat_ws('#','one') +-- !query analysis +Project [concat_ws(#, one) AS concat_ws(#, one)#x] ++- OneRowRelation + + +-- !query +select concat_ws('#',1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')) +-- !query analysis +Project [concat_ws(#, cast(1 as string), cast(2 as string), cast(3 as string), hello, cast(true as string), cast(false as string), cast(to_date(20100309, Some(yyyyMMdd), Some(America/Los_Angeles)) as string)) AS concat_ws(#, 1, 2, 3, hello, true, false, to_date(20100309, yyyyMMdd))#x] ++- OneRowRelation + + +-- !query +select concat_ws(',',10,20,null,30) +-- !query analysis +Project [concat_ws(,, cast(10 as string), cast(20 as string), cast(null as array), cast(30 as string)) AS concat_ws(,, 10, 20, NULL, 30)#x] ++- OneRowRelation + + +-- !query +select concat_ws('',10,20,null,30) +-- !query analysis +Project [concat_ws(, cast(10 as string), cast(20 as string), cast(null as array), cast(30 as string)) AS concat_ws(, 10, 20, NULL, 30)#x] ++- OneRowRelation + + +-- !query +select concat_ws(NULL,10,20,null,30) is null +-- !query analysis +Project [isnull(concat_ws(cast(null as string), cast(10 as string), cast(20 as string), cast(null as array), cast(30 as string))) AS (concat_ws(NULL, 10, 20, NULL, 30) IS NULL)#x] ++- OneRowRelation + + +-- !query +select reverse('abcde') +-- !query analysis +Project [reverse(abcde) AS reverse(abcde)#x] ++- OneRowRelation + + +-- !query +select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i +-- !query analysis +Sort [i#xL ASC NULLS FIRST], true ++- Project [i#xL, left(ahoj, cast(i#xL as int)) AS left(ahoj, i)#x, right(ahoj, cast(i#xL as int)) AS right(ahoj, i)#x] + +- SubqueryAlias t + +- Project [id#xL AS i#xL] + +- Range (-5, 6, step=1, splits=None) + + +-- !query +/* + * format + */ +select format_string(NULL) +-- !query analysis +Project [format_string(cast(null as string)) AS format_string(NULL)#x] ++- OneRowRelation + + +-- !query +select format_string('Hello') +-- !query analysis +Project [format_string(Hello) AS format_string(Hello)#x] ++- OneRowRelation + + +-- !query +select format_string('Hello %s', 'World') +-- !query analysis +Project [format_string(Hello %s, World) AS format_string(Hello %s, World)#x] ++- OneRowRelation + + +-- !query +select format_string('Hello %%') +-- !query analysis +Project [format_string(Hello %%) AS format_string(Hello %%)#x] ++- OneRowRelation + + +-- !query +select format_string('Hello %%%%') +-- !query analysis +Project [format_string(Hello %%%%) AS format_string(Hello %%%%)#x] ++- OneRowRelation + + +-- !query +select format_string('Hello %s %s', 'World') +-- !query analysis +Project [format_string(Hello %s %s, World) AS format_string(Hello %s %s, World)#x] ++- OneRowRelation + + +-- !query +select format_string('Hello %s') +-- !query analysis +Project [format_string(Hello %s) AS format_string(Hello %s)#x] ++- OneRowRelation + + +-- !query +select format_string('Hello %x', 20) +-- !query analysis +Project [format_string(Hello %x, 20) AS format_string(Hello %x, 20)#x] ++- OneRowRelation + + +-- !query +select format_string('%1$s %3$s', 1, 2, 3) +-- !query analysis +Project [format_string(%1$s %3$s, 1, 2, 3) AS format_string(%1$s %3$s, 1, 2, 3)#x] ++- OneRowRelation + + +-- !query +select format_string('%1$s %12$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) +-- !query analysis +Project [format_string(%1$s %12$s, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) AS format_string(%1$s %12$s, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)#x] ++- OneRowRelation + + +-- !query +select format_string('%1$s %4$s', 1, 2, 3) +-- !query analysis +Project [format_string(%1$s %4$s, 1, 2, 3) AS format_string(%1$s %4$s, 1, 2, 3)#x] ++- OneRowRelation + + +-- !query +select format_string('%1$s %13$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) +-- !query analysis +Project [format_string(%1$s %13$s, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) AS format_string(%1$s %13$s, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)#x] ++- OneRowRelation + + +-- !query +select format_string('%0$s', 'Hello') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.ZERO_INDEX", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`format_string`", + "parameter" : "`strfmt`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "format_string('%0$s', 'Hello')" + } ] +} + + +-- !query +select format_string('Hello %s %1$s %s', 'World', 'Hello again') +-- !query analysis +Project [format_string(Hello %s %1$s %s, World, Hello again) AS format_string(Hello %s %1$s %s, World, Hello again)#x] ++- OneRowRelation + + +-- !query +select format_string('Hello %s %s, %2$s %2$s', 'World', 'Hello again') +-- !query analysis +Project [format_string(Hello %s %s, %2$s %2$s, World, Hello again) AS format_string(Hello %s %s, %2$s %2$s, World, Hello again)#x] ++- OneRowRelation + + +-- !query +select format_string('>>%10s<<', 'Hello') +-- !query analysis +Project [format_string(>>%10s<<, Hello) AS format_string(>>%10s<<, Hello)#x] ++- OneRowRelation + + +-- !query +select format_string('>>%10s<<', NULL) +-- !query analysis +Project [format_string(>>%10s<<, null) AS format_string(>>%10s<<, NULL)#x] ++- OneRowRelation + + +-- !query +select format_string('>>%10s<<', '') +-- !query analysis +Project [format_string(>>%10s<<, ) AS format_string(>>%10s<<, )#x] ++- OneRowRelation + + +-- !query +select format_string('>>%-10s<<', '') +-- !query analysis +Project [format_string(>>%-10s<<, ) AS format_string(>>%-10s<<, )#x] ++- OneRowRelation + + +-- !query +select format_string('>>%-10s<<', 'Hello') +-- !query analysis +Project [format_string(>>%-10s<<, Hello) AS format_string(>>%-10s<<, Hello)#x] ++- OneRowRelation + + +-- !query +select format_string('>>%-10s<<', NULL) +-- !query analysis +Project [format_string(>>%-10s<<, null) AS format_string(>>%-10s<<, NULL)#x] ++- OneRowRelation + + +-- !query +select format_string('>>%1$10s<<', 'Hello') +-- !query analysis +Project [format_string(>>%1$10s<<, Hello) AS format_string(>>%1$10s<<, Hello)#x] ++- OneRowRelation + + +-- !query +DROP TABLE TEXT_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`TEXT_TBL`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/timestamp.sql.out new file mode 100644 index 0000000000000..2feefcf9745d6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/timestamp.sql.out @@ -0,0 +1,223 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`TIMESTAMP_TBL`, false + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'now') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1] ++- Project [cast(col1#x as timestamp) AS d1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'now') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1] ++- Project [cast(col1#x as timestamp) AS d1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'today') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1] ++- Project [cast(col1#x as timestamp) AS d1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'yesterday') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1] ++- Project [cast(col1#x as timestamp) AS d1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'tomorrow') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1] ++- Project [cast(col1#x as timestamp) AS d1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'tomorrow EST') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1] ++- Project [cast(col1#x as timestamp) AS d1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'tomorrow Zulu') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1] ++- Project [cast(col1#x as timestamp) AS d1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'today' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp 'tomorrow' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'yesterday' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +TRUNCATE TABLE TIMESTAMP_TBL +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`timestamp_tbl` + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'epoch') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1] ++- Project [cast(col1#x as timestamp) AS d1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1] ++- Project [cast(col1#x as timestamp) AS d1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02 03:04:05')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1] ++- Project [cast(col1#x as timestamp) AS d1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-02-10 17:32:01-08')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1] ++- Project [cast(col1#x as timestamp) AS d1#x] + +- LocalRelation [col1#x] + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('2001-09-22T18:19:20')) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1] ++- Project [cast(col1#x as timestamp) AS d1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT '' AS `64`, d1 FROM TIMESTAMP_TBL +-- !query analysis +Project [ AS 64#x, d1#x] ++- SubqueryAlias spark_catalog.default.timestamp_tbl + +- Relation spark_catalog.default.timestamp_tbl[d1#x] parquet + + +-- !query +SELECT '' AS `48`, d1 FROM TIMESTAMP_TBL + WHERE d1 > timestamp '1997-01-02' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT '' AS `15`, d1 FROM TIMESTAMP_TBL + WHERE d1 < timestamp '1997-01-02' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT '' AS one, d1 FROM TIMESTAMP_TBL + WHERE d1 = timestamp '1997-01-02' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT '' AS `63`, d1 FROM TIMESTAMP_TBL + WHERE d1 != timestamp '1997-01-02' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT '' AS `16`, d1 FROM TIMESTAMP_TBL + WHERE d1 <= timestamp '1997-01-02' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT '' AS `49`, d1 FROM TIMESTAMP_TBL + WHERE d1 >= timestamp '1997-01-02' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff + FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT '' AS date_trunc_week, date_trunc( 'week', timestamp '2004-02-29 15:44:17.71393' ) AS week_trunc +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff + FROM TIMESTAMP_TBL + WHERE d1 BETWEEN timestamp '1902-01-01' + AND timestamp '2038-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT '' AS `54`, d1 as `timestamp`, + date_part( 'year', d1) AS `year`, date_part( 'month', d1) AS `month`, + date_part( 'day', d1) AS `day`, date_part( 'hour', d1) AS `hour`, + date_part( 'minute', d1) AS `minute`, date_part( 'second', d1) AS `second` + FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01' +-- !query analysis +Project [ AS 54#x, d1#x AS timestamp#x, date_part(year, d1#x) AS year#x, date_part(month, d1#x) AS month#x, date_part(day, d1#x) AS day#x, date_part(hour, d1#x) AS hour#x, date_part(minute, d1#x) AS minute#x, date_part(second, d1#x) AS second#x] ++- Filter ((d1#x >= cast(1902-01-01 as timestamp)) AND (d1#x <= cast(2038-01-01 as timestamp))) + +- SubqueryAlias spark_catalog.default.timestamp_tbl + +- Relation spark_catalog.default.timestamp_tbl[d1#x] parquet + + +-- !query +SELECT make_timestamp(2014,12,28,6,30,45.887) +-- !query analysis +Project [make_timestamp(2014, 12, 28, 6, 30, cast(45.887 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(2014, 12, 28, 6, 30, 45.887)#x] ++- OneRowRelation + + +-- !query +DROP TABLE TIMESTAMP_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`TIMESTAMP_TBL`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/union.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/union.sql.out new file mode 100644 index 0000000000000..f08678bf93b57 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/union.sql.out @@ -0,0 +1,1326 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM + (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) + AS v(f1) +-- !query analysis +CreateViewCommand `INT4_TBL`, SELECT * FROM + (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) + AS v(f1), false, true, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2) +-- !query analysis +CreateViewCommand `INT8_TBL`, SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2), false, true, LocalTempView, true + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM + (VALUES (0.0), (-34.84), (-1004.30), + (CAST('-1.2345678901234e+200' AS DOUBLE)), (CAST('-1.2345678901234e-200' AS DOUBLE))) + AS v(f1) +-- !query analysis +CreateViewCommand `FLOAT8_TBL`, SELECT * FROM + (VALUES (0.0), (-34.84), (-1004.30), + (CAST('-1.2345678901234e+200' AS DOUBLE)), (CAST('-1.2345678901234e-200' AS DOUBLE))) + AS v(f1), false, true, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT 1 AS two UNION SELECT 2 ORDER BY 1 +-- !query analysis +Sort [two#x ASC NULLS FIRST], true ++- Distinct + +- Union false, false + :- Project [1 AS two#x] + : +- OneRowRelation + +- Project [2 AS 2#x] + +- OneRowRelation + + +-- !query +SELECT 1 AS one UNION SELECT 1 ORDER BY 1 +-- !query analysis +Sort [one#x ASC NULLS FIRST], true ++- Distinct + +- Union false, false + :- Project [1 AS one#x] + : +- OneRowRelation + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT 1 AS two UNION ALL SELECT 2 +-- !query analysis +Union false, false +:- Project [1 AS two#x] +: +- OneRowRelation ++- Project [2 AS 2#x] + +- OneRowRelation + + +-- !query +SELECT 1 AS two UNION ALL SELECT 1 +-- !query analysis +Union false, false +:- Project [1 AS two#x] +: +- OneRowRelation ++- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT 1 AS three UNION SELECT 2 UNION SELECT 3 ORDER BY 1 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELECT'", + "hint" : "" + } +} + + +-- !query +SELECT 1 AS two UNION SELECT 2 UNION SELECT 2 ORDER BY 1 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELECT'", + "hint" : "" + } +} + + +-- !query +SELECT 1 AS three UNION SELECT 2 UNION ALL SELECT 2 ORDER BY 1 +-- !query analysis +Sort [three#x ASC NULLS FIRST], true ++- Union false, false + :- Distinct + : +- Union false, false + : :- Project [1 AS three#x] + : : +- OneRowRelation + : +- Project [2 AS 2#x] + : +- OneRowRelation + +- Project [2 AS 2#x] + +- OneRowRelation + + +-- !query +SELECT 1.1 AS two UNION SELECT 2.2 ORDER BY 1 +-- !query analysis +Sort [two#x ASC NULLS FIRST], true ++- Distinct + +- Union false, false + :- Project [1.1 AS two#x] + : +- OneRowRelation + +- Project [2.2 AS 2.2#x] + +- OneRowRelation + + +-- !query +SELECT 1.1 AS two UNION SELECT 2 ORDER BY 1 +-- !query analysis +Sort [two#x ASC NULLS FIRST], true ++- Distinct + +- Union false, false + :- Project [cast(two#x as decimal(11,1)) AS two#x] + : +- Project [1.1 AS two#x] + : +- OneRowRelation + +- Project [cast(2#x as decimal(11,1)) AS 2#x] + +- Project [2 AS 2#x] + +- OneRowRelation + + +-- !query +SELECT 1 AS two UNION SELECT 2.2 ORDER BY 1 +-- !query analysis +Sort [two#x ASC NULLS FIRST], true ++- Distinct + +- Union false, false + :- Project [cast(two#x as decimal(11,1)) AS two#x] + : +- Project [1 AS two#x] + : +- OneRowRelation + +- Project [cast(2.2#x as decimal(11,1)) AS 2.2#x] + +- Project [2.2 AS 2.2#x] + +- OneRowRelation + + +-- !query +SELECT 1 AS one UNION SELECT double(1.0) ORDER BY 1 +-- !query analysis +Sort [one#x ASC NULLS FIRST], true ++- Distinct + +- Union false, false + :- Project [cast(one#x as double) AS one#x] + : +- Project [1 AS one#x] + : +- OneRowRelation + +- Project [cast(1.0 as double) AS 1.0#x] + +- OneRowRelation + + +-- !query +SELECT 1.1 AS two UNION ALL SELECT 2 ORDER BY 1 +-- !query analysis +Sort [two#x ASC NULLS FIRST], true ++- Union false, false + :- Project [cast(two#x as decimal(11,1)) AS two#x] + : +- Project [1.1 AS two#x] + : +- OneRowRelation + +- Project [cast(2#x as decimal(11,1)) AS 2#x] + +- Project [2 AS 2#x] + +- OneRowRelation + + +-- !query +SELECT double(1.0) AS two UNION ALL SELECT 1 ORDER BY 1 +-- !query analysis +Sort [two#x ASC NULLS FIRST], true ++- Union false, false + :- Project [cast(1.0 as double) AS two#x] + : +- OneRowRelation + +- Project [cast(1#x as double) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT 1.1 AS three UNION SELECT 2 UNION SELECT 3 ORDER BY 1 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELECT'", + "hint" : "" + } +} + + +-- !query +SELECT double(1.1) AS two UNION SELECT 2 UNION SELECT double(2.0) ORDER BY 1 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELECT'", + "hint" : "" + } +} + + +-- !query +SELECT 1.1 AS three UNION SELECT 2 UNION ALL SELECT 2 ORDER BY 1 +-- !query analysis +Sort [three#x ASC NULLS FIRST], true ++- Union false, false + :- Distinct + : +- Union false, false + : :- Project [cast(three#x as decimal(11,1)) AS three#x] + : : +- Project [1.1 AS three#x] + : : +- OneRowRelation + : +- Project [cast(2#x as decimal(11,1)) AS 2#x] + : +- Project [2 AS 2#x] + : +- OneRowRelation + +- Project [cast(2#x as decimal(11,1)) AS 2#x] + +- Project [2 AS 2#x] + +- OneRowRelation + + +-- !query +SELECT 1.1 AS two UNION (SELECT 2 UNION ALL SELECT 2) ORDER BY 1 +-- !query analysis +Sort [two#x ASC NULLS FIRST], true ++- Distinct + +- Union false, false + :- Project [cast(two#x as decimal(11,1)) AS two#x] + : +- Project [1.1 AS two#x] + : +- OneRowRelation + +- Project [cast(2#x as decimal(11,1)) AS 2#x] + +- Union false, false + :- Project [2 AS 2#x] + : +- OneRowRelation + +- Project [2 AS 2#x] + +- OneRowRelation + + +-- !query +SELECT f1 AS five FROM FLOAT8_TBL +UNION +SELECT f1 FROM FLOAT8_TBL +ORDER BY 1 +-- !query analysis +Sort [five#x ASC NULLS FIRST], true ++- Distinct + +- Union false, false + :- Project [f1#x AS five#x] + : +- SubqueryAlias float8_tbl + : +- View (`FLOAT8_TBL`, [f1#x]) + : +- Project [cast(f1#x as double) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- Project [f1#x] + +- SubqueryAlias float8_tbl + +- View (`FLOAT8_TBL`, [f1#x]) + +- Project [cast(f1#x as double) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT f1 AS ten FROM FLOAT8_TBL +UNION ALL +SELECT f1 FROM FLOAT8_TBL +-- !query analysis +Union false, false +:- Project [f1#x AS ten#x] +: +- SubqueryAlias float8_tbl +: +- View (`FLOAT8_TBL`, [f1#x]) +: +- Project [cast(f1#x as double) AS f1#x] +: +- Project [f1#x] +: +- SubqueryAlias v +: +- Project [col1#x AS f1#x] +: +- LocalRelation [col1#x] ++- Project [f1#x] + +- SubqueryAlias float8_tbl + +- View (`FLOAT8_TBL`, [f1#x]) + +- Project [cast(f1#x as double) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT f1 AS nine FROM FLOAT8_TBL +UNION +SELECT f1 FROM INT4_TBL +ORDER BY 1 +-- !query analysis +Sort [nine#x ASC NULLS FIRST], true ++- Distinct + +- Union false, false + :- Project [f1#x AS nine#x] + : +- SubqueryAlias float8_tbl + : +- View (`FLOAT8_TBL`, [f1#x]) + : +- Project [cast(f1#x as double) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- Project [cast(f1#x as double) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT f1 AS ten FROM FLOAT8_TBL +UNION ALL +SELECT f1 FROM INT4_TBL +-- !query analysis +Union false, false +:- Project [f1#x AS ten#x] +: +- SubqueryAlias float8_tbl +: +- View (`FLOAT8_TBL`, [f1#x]) +: +- Project [cast(f1#x as double) AS f1#x] +: +- Project [f1#x] +: +- SubqueryAlias v +: +- Project [col1#x AS f1#x] +: +- LocalRelation [col1#x] ++- Project [cast(f1#x as double) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT f1 AS five FROM FLOAT8_TBL + WHERE f1 BETWEEN -1e6 AND 1e6 +UNION +SELECT f1 FROM INT4_TBL + WHERE f1 BETWEEN 0 AND 1000000 +ORDER BY 1 +-- !query analysis +Sort [five#x ASC NULLS FIRST], true ++- Distinct + +- Union false, false + :- Project [f1#x AS five#x] + : +- Filter ((f1#x >= -1000000.0) AND (f1#x <= 1000000.0)) + : +- SubqueryAlias float8_tbl + : +- View (`FLOAT8_TBL`, [f1#x]) + : +- Project [cast(f1#x as double) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- Project [cast(f1#x as double) AS f1#x] + +- Project [f1#x] + +- Filter ((f1#x >= 0) AND (f1#x <= 1000000)) + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT q2 FROM int8_tbl INTERSECT SELECT q1 FROM int8_tbl ORDER BY 1 +-- !query analysis +Sort [q2#xL ASC NULLS FIRST], true ++- Intersect false + :- Project [q2#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Project [q1#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +SELECT q2 FROM int8_tbl INTERSECT ALL SELECT q1 FROM int8_tbl ORDER BY 1 +-- !query analysis +Sort [q2#xL ASC NULLS FIRST], true ++- Intersect All true + :- Project [q2#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Project [q1#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +SELECT q2 FROM int8_tbl EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1 +-- !query analysis +Sort [q2#xL ASC NULLS FIRST], true ++- Except false + :- Project [q2#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Project [q1#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +SELECT q2 FROM int8_tbl EXCEPT ALL SELECT q1 FROM int8_tbl ORDER BY 1 +-- !query analysis +Sort [q2#xL ASC NULLS FIRST], true ++- Except All true + :- Project [q2#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Project [q1#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +SELECT q2 FROM int8_tbl EXCEPT ALL SELECT DISTINCT q1 FROM int8_tbl ORDER BY 1 +-- !query analysis +Sort [q2#xL ASC NULLS FIRST], true ++- Except All true + :- Project [q2#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Distinct + +- Project [q1#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +SELECT q1 FROM int8_tbl EXCEPT SELECT q2 FROM int8_tbl ORDER BY 1 +-- !query analysis +Sort [q1#xL ASC NULLS FIRST], true ++- Except false + :- Project [q1#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Project [q2#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +SELECT q1 FROM int8_tbl EXCEPT ALL SELECT q2 FROM int8_tbl ORDER BY 1 +-- !query analysis +Sort [q1#xL ASC NULLS FIRST], true ++- Except All true + :- Project [q1#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Project [q2#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +SELECT q1 FROM int8_tbl EXCEPT ALL SELECT DISTINCT q2 FROM int8_tbl ORDER BY 1 +-- !query analysis +Sort [q1#xL ASC NULLS FIRST], true ++- Except All true + :- Project [q1#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Distinct + +- Project [q2#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +(SELECT 1,2,3 UNION SELECT 4,5,6) INTERSECT SELECT 4,5,6 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELECT'", + "hint" : "" + } +} + + +-- !query +(SELECT 1,2,3 UNION SELECT 4,5,6 ORDER BY 1,2) INTERSECT SELECT 4,5,6 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELECT'", + "hint" : "" + } +} + + +-- !query +(SELECT 1,2,3 UNION SELECT 4,5,6) EXCEPT SELECT 4,5,6 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELECT'", + "hint" : "" + } +} + + +-- !query +(SELECT 1,2,3 UNION SELECT 4,5,6 ORDER BY 1,2) EXCEPT SELECT 4,5,6 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELECT'", + "hint" : "" + } +} + + +-- !query +select count(*) from + ( select unique1 from tenk1 intersect select fivethous from tenk1 ) ss +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- SubqueryAlias ss + +- Intersect false + :- Project [unique1#x] + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- Project [fivethous#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select unique1 from tenk1 except select unique2 from tenk1 where unique2 != 10 +-- !query analysis +Except false +:- Project [unique1#x] +: +- SubqueryAlias spark_catalog.default.tenk1 +: +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet ++- Project [unique2#x] + +- Filter NOT (unique2#x = 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select count(*) from + ( select unique1 from tenk1 intersect select fivethous from tenk1 ) ss +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- SubqueryAlias ss + +- Intersect false + :- Project [unique1#x] + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- Project [fivethous#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select unique1 from tenk1 except select unique2 from tenk1 where unique2 != 10 +-- !query analysis +Except false +:- Project [unique1#x] +: +- SubqueryAlias spark_catalog.default.tenk1 +: +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet ++- Project [unique2#x] + +- Filter NOT (unique2#x = 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT f1 FROM float8_tbl INTERSECT SELECT f1 FROM int4_tbl ORDER BY 1 +-- !query analysis +Sort [f1#x ASC NULLS FIRST], true ++- Intersect false + :- Project [f1#x] + : +- SubqueryAlias float8_tbl + : +- View (`FLOAT8_TBL`, [f1#x]) + : +- Project [cast(f1#x as double) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- Project [cast(f1#x as double) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT f1 FROM float8_tbl EXCEPT SELECT f1 FROM int4_tbl ORDER BY 1 +-- !query analysis +Sort [f1#x ASC NULLS FIRST], true ++- Except false + :- Project [f1#x] + : +- SubqueryAlias float8_tbl + : +- View (`FLOAT8_TBL`, [f1#x]) + : +- Project [cast(f1#x as double) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- Project [cast(f1#x as double) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT q1 FROM int8_tbl INTERSECT SELECT q2 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl ORDER BY 1 +-- !query analysis +Sort [q1#xL ASC NULLS FIRST], true ++- Union false, false + :- Intersect false + : :- Project [q1#xL] + : : +- SubqueryAlias int8_tbl + : : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : : +- Project [q1#xL, q2#xL] + : : +- SubqueryAlias v + : : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : : +- LocalRelation [col1#xL, col2#xL] + : +- Project [q2#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Project [q2#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +SELECT q1 FROM int8_tbl INTERSECT (((SELECT q2 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl))) ORDER BY 1 +-- !query analysis +Sort [q1#xL ASC NULLS FIRST], true ++- Intersect false + :- Project [q1#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Union false, false + :- Project [q2#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Project [q2#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +(((SELECT q1 FROM int8_tbl INTERSECT SELECT q2 FROM int8_tbl ORDER BY 1))) UNION ALL SELECT q2 FROM int8_tbl +-- !query analysis +Union false, false +:- Sort [q1#xL ASC NULLS FIRST], true +: +- Intersect false +: :- Project [q1#xL] +: : +- SubqueryAlias int8_tbl +: : +- View (`INT8_TBL`, [q1#xL,q2#xL]) +: : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] +: : +- Project [q1#xL, q2#xL] +: : +- SubqueryAlias v +: : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] +: : +- LocalRelation [col1#xL, col2#xL] +: +- Project [q2#xL] +: +- SubqueryAlias int8_tbl +: +- View (`INT8_TBL`, [q1#xL,q2#xL]) +: +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] +: +- Project [q1#xL, q2#xL] +: +- SubqueryAlias v +: +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] +: +- LocalRelation [col1#xL, col2#xL] ++- Project [q2#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +SELECT q1 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1 +-- !query analysis +Sort [q1#xL ASC NULLS FIRST], true ++- Except false + :- Union false, false + : :- Project [q1#xL] + : : +- SubqueryAlias int8_tbl + : : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : : +- Project [q1#xL, q2#xL] + : : +- SubqueryAlias v + : : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : : +- LocalRelation [col1#xL, col2#xL] + : +- Project [q2#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Project [q1#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +SELECT q1 FROM int8_tbl UNION ALL (((SELECT q2 FROM int8_tbl EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1))) +-- !query analysis +Union false, false +:- Project [q1#xL] +: +- SubqueryAlias int8_tbl +: +- View (`INT8_TBL`, [q1#xL,q2#xL]) +: +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] +: +- Project [q1#xL, q2#xL] +: +- SubqueryAlias v +: +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] +: +- LocalRelation [col1#xL, col2#xL] ++- Sort [q2#xL ASC NULLS FIRST], true + +- Except false + :- Project [q2#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Project [q1#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +(((SELECT q1 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl))) EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1 +-- !query analysis +Sort [q1#xL ASC NULLS FIRST], true ++- Except false + :- Union false, false + : :- Project [q1#xL] + : : +- SubqueryAlias int8_tbl + : : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : : +- Project [q1#xL, q2#xL] + : : +- SubqueryAlias v + : : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : : +- LocalRelation [col1#xL, col2#xL] + : +- Project [q2#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Project [q1#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +SELECT q1,q2 FROM int8_tbl EXCEPT SELECT q2,q1 FROM int8_tbl +ORDER BY q2,q1 +-- !query analysis +Sort [q2#xL ASC NULLS FIRST, q1#xL ASC NULLS FIRST], true ++- Except false + :- Project [q1#xL, q2#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Project [q2#xL, q1#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +SELECT q1 FROM int8_tbl EXCEPT SELECT q2 FROM int8_tbl ORDER BY q2 LIMIT 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`q2`", + "proposal" : "`int8_tbl`.`q1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 65, + "stopIndex" : 66, + "fragment" : "q2" + } ] +} + + +-- !query +SELECT q1 FROM int8_tbl EXCEPT (((SELECT q2 FROM int8_tbl ORDER BY q2 LIMIT 1))) ORDER BY 1 +-- !query analysis +Sort [q1#xL ASC NULLS FIRST], true ++- Except false + :- Project [q1#xL] + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- GlobalLimit 1 + +- LocalLimit 1 + +- Sort [q2#xL ASC NULLS FIRST], true + +- Project [q2#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +(((((select * from int8_tbl))))) +-- !query analysis +Project [q1#xL, q2#xL] ++- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select * from range(1,5) union select * from range(1,3) +-- !query analysis +Distinct ++- Union false, false + :- Project [id#xL] + : +- Range (1, 5, step=1, splits=None) + +- Project [id#xL] + +- Range (1, 3, step=1, splits=None) + + +-- !query +select * from range(1,6) union all select * from range(1,4) +-- !query analysis +Union false, false +:- Project [id#xL] +: +- Range (1, 6, step=1, splits=None) ++- Project [id#xL] + +- Range (1, 4, step=1, splits=None) + + +-- !query +select * from range(1,6) intersect select * from range(1,4) +-- !query analysis +Intersect false +:- Project [id#xL] +: +- Range (1, 6, step=1, splits=None) ++- Project [id#xL] + +- Range (1, 4, step=1, splits=None) + + +-- !query +select * from range(1,6) intersect all select * from range(1,4) +-- !query analysis +Intersect All true +:- Project [id#xL] +: +- Range (1, 6, step=1, splits=None) ++- Project [id#xL] + +- Range (1, 4, step=1, splits=None) + + +-- !query +select * from range(1,6) except select * from range(1,4) +-- !query analysis +Except false +:- Project [id#xL] +: +- Range (1, 6, step=1, splits=None) ++- Project [id#xL] + +- Range (1, 4, step=1, splits=None) + + +-- !query +select * from range(1,6) except all select * from range(1,4) +-- !query analysis +Except All true +:- Project [id#xL] +: +- Range (1, 6, step=1, splits=None) ++- Project [id#xL] + +- Range (1, 4, step=1, splits=None) + + +-- !query +select * from range(1,6) union select * from range(1,4) +-- !query analysis +Distinct ++- Union false, false + :- Project [id#xL] + : +- Range (1, 6, step=1, splits=None) + +- Project [id#xL] + +- Range (1, 4, step=1, splits=None) + + +-- !query +select * from range(1,6) union all select * from range(1,4) +-- !query analysis +Union false, false +:- Project [id#xL] +: +- Range (1, 6, step=1, splits=None) ++- Project [id#xL] + +- Range (1, 4, step=1, splits=None) + + +-- !query +select * from range(1,6) intersect select * from range(1,4) +-- !query analysis +Intersect false +:- Project [id#xL] +: +- Range (1, 6, step=1, splits=None) ++- Project [id#xL] + +- Range (1, 4, step=1, splits=None) + + +-- !query +select * from range(1,6) intersect all select * from range(1,4) +-- !query analysis +Intersect All true +:- Project [id#xL] +: +- Range (1, 6, step=1, splits=None) ++- Project [id#xL] + +- Range (1, 4, step=1, splits=None) + + +-- !query +select * from range(1,6) except select * from range(1,4) +-- !query analysis +Except false +:- Project [id#xL] +: +- Range (1, 6, step=1, splits=None) ++- Project [id#xL] + +- Range (1, 4, step=1, splits=None) + + +-- !query +select * from range(1,6) except all select * from range(1,4) +-- !query analysis +Except All true +:- Project [id#xL] +: +- Range (1, 6, step=1, splits=None) ++- Project [id#xL] + +- Range (1, 4, step=1, splits=None) + + +-- !query +SELECT cast('3.4' as decimal(38, 18)) UNION SELECT 'foo' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELECT'", + "hint" : "" + } +} + + +-- !query +SELECT * FROM + (SELECT 1 AS t, 2 AS x + UNION + SELECT 2 AS t, 4 AS x) ss +WHERE x < 4 +ORDER BY x +-- !query analysis +Sort [x#x ASC NULLS FIRST], true ++- Project [t#x, x#x] + +- Filter (x#x < 4) + +- SubqueryAlias ss + +- Distinct + +- Union false, false + :- Project [1 AS t#x, 2 AS x#x] + : +- OneRowRelation + +- Project [2 AS t#x, 4 AS x#x] + +- OneRowRelation + + +-- !query +SELECT * FROM + (SELECT 1 AS t, id as x from range(1,11) + UNION + SELECT 2 AS t, 4 AS x) ss +WHERE x < 4 +ORDER BY x +-- !query analysis +Sort [x#xL ASC NULLS FIRST], true ++- Project [t#x, x#xL] + +- Filter (x#xL < cast(4 as bigint)) + +- SubqueryAlias ss + +- Distinct + +- Union false, false + :- Project [1 AS t#x, id#xL AS x#xL] + : +- Range (1, 11, step=1, splits=None) + +- Project [t#x, cast(x#x as bigint) AS x#xL] + +- Project [2 AS t#x, 4 AS x#x] + +- OneRowRelation + + +-- !query +SELECT * FROM + (SELECT 1 AS t, int((random()*3)) AS x + UNION + SELECT 2 AS t, 4 AS x) ss +WHERE x > 3 +ORDER BY x +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select distinct q1 from + (select distinct * from int8_tbl i81 + union all + select distinct * from int8_tbl i82) ss +where q2 = q2 +-- !query analysis +Distinct ++- Project [q1#xL] + +- Filter (q2#xL = q2#xL) + +- SubqueryAlias ss + +- Union false, false + :- Distinct + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias i81 + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Distinct + +- Project [q1#xL, q2#xL] + +- SubqueryAlias i82 + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select distinct q1 from + (select distinct * from int8_tbl i81 + union all + select distinct * from int8_tbl i82) ss +where -q1 = q2 +-- !query analysis +Distinct ++- Project [q1#xL] + +- Filter (-q1#xL = q2#xL) + +- SubqueryAlias ss + +- Union false, false + :- Distinct + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias i81 + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Distinct + +- Project [q1#xL, q2#xL] + +- SubqueryAlias i82 + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select * from + (select *, 0 as x from int8_tbl a + union all + select *, 1 as x from int8_tbl b) ss +where (x = 0) or (q1 >= q2 and q1 <= q2) +-- !query analysis +Project [q1#xL, q2#xL, x#x] ++- Filter ((x#x = 0) OR ((q1#xL >= q2#xL) AND (q1#xL <= q2#xL))) + +- SubqueryAlias ss + +- Union false, false + :- Project [q1#xL, q2#xL, 0 AS x#x] + : +- SubqueryAlias a + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Project [q1#xL, q2#xL, 1 AS x#x] + +- SubqueryAlias b + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part1.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part1.sql.out new file mode 100644 index 0000000000000..1e4848eab64b2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part1.sql.out @@ -0,0 +1,564 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 +-- !query analysis +CreateViewCommand `tenk2`, SELECT * FROM tenk1, false, false, LocalTempView, true + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT four, ten, SUM(SUM(four)) OVER (PARTITION BY four), AVG(ten) FROM tenk1 +GROUP BY four, ten ORDER BY four, ten +-- !query analysis +Sort [four#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true ++- Project [four#x, ten#x, sum(sum(four)) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, avg(ten)#x] + +- Project [four#x, ten#x, avg(ten)#x, _w0#xL, sum(sum(four)) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, sum(sum(four)) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] + +- Window [sum(_w0#xL) windowspecdefinition(four#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS sum(sum(four)) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL], [four#x] + +- Aggregate [four#x, ten#x], [four#x, ten#x, avg(ten#x) AS avg(ten)#x, sum(four#x) AS _w0#xL] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT COUNT(*) OVER () FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [count(1) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] ++- Project [count(1) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, count(1) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] + +- Window [count(1) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS count(1) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] + +- Project + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT COUNT(*) OVER w FROM tenk1 WHERE unique2 < 10 WINDOW w AS () +-- !query analysis +Project [count(1) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] ++- Project [count(1) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, count(1) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] + +- Window [count(1) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS count(1) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] + +- Project + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT four FROM tenk1 WHERE FALSE WINDOW w AS (PARTITION BY ten) +-- !query analysis +Project [four#x] ++- Filter false + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT sum(four) OVER (PARTITION BY ten ORDER BY unique2) AS sum_1, ten, four FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [sum_1#xL, ten#x, four#x] ++- Project [ten#x, four#x, unique2#x, sum_1#xL, sum_1#xL] + +- Window [sum(four#x) windowspecdefinition(ten#x, unique2#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS sum_1#xL], [ten#x], [unique2#x ASC NULLS FIRST] + +- Project [ten#x, four#x, unique2#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT row_number() OVER (ORDER BY unique2) FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [row_number() OVER (ORDER BY unique2 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [unique2#x, row_number() OVER (ORDER BY unique2 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, row_number() OVER (ORDER BY unique2 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [row_number() windowspecdefinition(unique2#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS row_number() OVER (ORDER BY unique2 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [unique2#x ASC NULLS FIRST] + +- Project [unique2#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT rank() OVER (PARTITION BY four ORDER BY ten) AS rank_1, ten, four FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [rank_1#x, ten#x, four#x] ++- Project [ten#x, four#x, rank_1#x, rank_1#x] + +- Window [rank(ten#x) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_1#x], [four#x], [ten#x ASC NULLS FIRST] + +- Project [ten#x, four#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT dense_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [DENSE_RANK() OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, ten#x, four#x] ++- Project [ten#x, four#x, DENSE_RANK() OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, DENSE_RANK() OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [dense_rank(ten#x) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS DENSE_RANK() OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [four#x], [ten#x ASC NULLS FIRST] + +- Project [ten#x, four#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT percent_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [PERCENT_RANK() OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, ten#x, four#x] ++- Project [ten#x, four#x, PERCENT_RANK() OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, PERCENT_RANK() OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [percent_rank(ten#x) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS PERCENT_RANK() OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [four#x], [ten#x ASC NULLS FIRST] + +- Project [ten#x, four#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT cume_dist() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [cume_dist() OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, ten#x, four#x] ++- Project [ten#x, four#x, cume_dist() OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, cume_dist() OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [cume_dist() windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS cume_dist() OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [four#x], [ten#x ASC NULLS FIRST] + +- Project [ten#x, four#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT ntile(3) OVER (ORDER BY ten, four), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [ntile(3) OVER (ORDER BY ten ASC NULLS FIRST, four ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, ten#x, four#x] ++- Project [ten#x, four#x, ntile(3) OVER (ORDER BY ten ASC NULLS FIRST, four ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, ntile(3) OVER (ORDER BY ten ASC NULLS FIRST, four ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [ntile(3) windowspecdefinition(ten#x ASC NULLS FIRST, four#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ntile(3) OVER (ORDER BY ten ASC NULLS FIRST, four ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [ten#x ASC NULLS FIRST, four#x ASC NULLS FIRST] + +- Project [ten#x, four#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT lag(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [lag(ten, 1, NULL) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN -1 FOLLOWING AND -1 FOLLOWING)#x, ten#x, four#x] ++- Project [ten#x, four#x, lag(ten, 1, NULL) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN -1 FOLLOWING AND -1 FOLLOWING)#x, lag(ten, 1, NULL) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN -1 FOLLOWING AND -1 FOLLOWING)#x] + +- Window [lag(ten#x, -1, null) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, -1)) AS lag(ten, 1, NULL) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN -1 FOLLOWING AND -1 FOLLOWING)#x], [four#x], [ten#x ASC NULLS FIRST] + +- Project [ten#x, four#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT lead(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [lead(ten, 1, NULL) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN 1 FOLLOWING AND 1 FOLLOWING)#x, ten#x, four#x] ++- Project [ten#x, four#x, lead(ten, 1, NULL) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN 1 FOLLOWING AND 1 FOLLOWING)#x, lead(ten, 1, NULL) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN 1 FOLLOWING AND 1 FOLLOWING)#x] + +- Window [lead(ten#x, 1, null) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 1, 1)) AS lead(ten, 1, NULL) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN 1 FOLLOWING AND 1 FOLLOWING)#x], [four#x], [ten#x ASC NULLS FIRST] + +- Project [ten#x, four#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT lead(ten * 2, 1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [lead((ten * 2), 1, NULL) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN 1 FOLLOWING AND 1 FOLLOWING)#x, ten#x, four#x] ++- Project [ten#x, four#x, _w0#x, lead((ten * 2), 1, NULL) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN 1 FOLLOWING AND 1 FOLLOWING)#x, lead((ten * 2), 1, NULL) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN 1 FOLLOWING AND 1 FOLLOWING)#x] + +- Window [lead(_w0#x, 1, null) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 1, 1)) AS lead((ten * 2), 1, NULL) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN 1 FOLLOWING AND 1 FOLLOWING)#x], [four#x], [ten#x ASC NULLS FIRST] + +- Project [ten#x, four#x, (ten#x * 2) AS _w0#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT lead(ten * 2, 1, -1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [lead((ten * 2), 1, -1) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN 1 FOLLOWING AND 1 FOLLOWING)#x, ten#x, four#x] ++- Project [ten#x, four#x, _w0#x, lead((ten * 2), 1, -1) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN 1 FOLLOWING AND 1 FOLLOWING)#x, lead((ten * 2), 1, -1) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN 1 FOLLOWING AND 1 FOLLOWING)#x] + +- Window [lead(_w0#x, 1, -1) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 1, 1)) AS lead((ten * 2), 1, -1) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST ROWS BETWEEN 1 FOLLOWING AND 1 FOLLOWING)#x], [four#x], [ten#x ASC NULLS FIRST] + +- Project [ten#x, four#x, (ten#x * 2) AS _w0#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT first(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [first(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, ten#x, four#x] ++- Project [ten#x, four#x, first(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, first(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [first(ten#x, false) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [four#x], [ten#x ASC NULLS FIRST] + +- Project [ten#x, four#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT last(four) OVER (ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [last(four) OVER (ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, ten#x, four#x] ++- Project [ten#x, four#x, last(four) OVER (ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, last(four) OVER (ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [last(four#x, false) windowspecdefinition(ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last(four) OVER (ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [ten#x ASC NULLS FIRST] + +- Project [ten#x, four#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT last(ten) OVER (PARTITION BY four), ten, four FROM +(SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s +ORDER BY four, ten +-- !query analysis +Sort [four#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true ++- Project [last(ten) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, ten#x, four#x] + +- Project [ten#x, four#x, last(ten) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, last(ten) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Window [last(ten#x, false) windowspecdefinition(four#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS last(ten) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x], [four#x] + +- Project [ten#x, four#x] + +- SubqueryAlias s + +- Sort [four#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER (PARTITION BY two ORDER BY ten) AS wsum +FROM tenk1 GROUP BY ten, two +-- !query analysis +Project [ten#x, two#x, gsum#xL, wsum#xL] ++- Project [ten#x, two#x, gsum#xL, _w0#xL, wsum#xL, wsum#xL] + +- Window [sum(_w0#xL) windowspecdefinition(two#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS wsum#xL], [two#x], [ten#x ASC NULLS FIRST] + +- Aggregate [ten#x, two#x], [ten#x, two#x, sum(hundred#x) AS gsum#xL, sum(hundred#x) AS _w0#xL] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT count(*) OVER (PARTITION BY four), four FROM (SELECT * FROM tenk1 WHERE two = 1)s WHERE unique2 < 10 +-- !query analysis +Project [count(1) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, four#x] ++- Project [four#x, count(1) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, count(1) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] + +- Window [count(1) windowspecdefinition(four#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS count(1) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL], [four#x] + +- Project [four#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias s + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- Filter (two#x = 1) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT (count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY four ORDER BY ten)) AS cntsum + FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [cntsum#xL] ++- Project [four#x, ten#x, hundred#x, _we0#xL, _we1#xL, (_we0#xL + _we1#xL) AS cntsum#xL] + +- Window [count(1) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS _we0#xL, sum(hundred#x) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS _we1#xL], [four#x], [ten#x ASC NULLS FIRST] + +- Project [four#x, ten#x, hundred#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT * FROM( + SELECT count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS total, + count(*) OVER (PARTITION BY four ORDER BY ten) AS fourcount, + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS twosum + FROM tenk1 +)sub WHERE total <> fourcount + twosum +-- !query analysis +Project [total#xL, fourcount#xL, twosum#xL] ++- Filter NOT (total#xL = (fourcount#xL + twosum#xL)) + +- SubqueryAlias sub + +- Project [total#xL, fourcount#xL, twosum#xL] + +- Project [four#x, ten#x, hundred#x, two#x, _we0#xL, fourcount#xL, _we1#xL, twosum#xL, (_we0#xL + _we1#xL) AS total#xL, fourcount#xL, twosum#xL] + +- Window [sum(hundred#x) windowspecdefinition(two#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS _we1#xL, sum(hundred#x) windowspecdefinition(two#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS twosum#xL], [two#x], [ten#x ASC NULLS FIRST] + +- Window [count(1) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS _we0#xL, count(1) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS fourcount#xL], [four#x], [ten#x ASC NULLS FIRST] + +- Project [four#x, ten#x, hundred#x, two#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT avg(four) OVER (PARTITION BY four ORDER BY thousand / 100) FROM tenk1 WHERE unique2 < 10 +-- !query analysis +Project [avg(four) OVER (PARTITION BY four ORDER BY (thousand / 100) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [four#x, _w1#x, avg(four) OVER (PARTITION BY four ORDER BY (thousand / 100) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, avg(four) OVER (PARTITION BY four ORDER BY (thousand / 100) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [avg(four#x) windowspecdefinition(four#x, _w1#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS avg(four) OVER (PARTITION BY four ORDER BY (thousand / 100) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [four#x], [_w1#x ASC NULLS FIRST] + +- Project [four#x, (cast(thousand#x as double) / cast(100 as double)) AS _w1#x] + +- Filter (unique2#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER win AS wsum +FROM tenk1 GROUP BY ten, two WINDOW win AS (PARTITION BY two ORDER BY ten) +-- !query analysis +Project [ten#x, two#x, gsum#xL, wsum#xL] ++- Project [ten#x, two#x, gsum#xL, _w0#xL, wsum#xL, wsum#xL] + +- Window [sum(_w0#xL) windowspecdefinition(two#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS wsum#xL], [two#x], [ten#x ASC NULLS FIRST] + +- Aggregate [ten#x, two#x], [ten#x, two#x, sum(hundred#x) AS gsum#xL, sum(hundred#x) AS _w0#xL] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 WHERE FALSE)s +-- !query analysis +Project [count(1) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] ++- Project [four#x, count(1) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, count(1) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] + +- Window [count(1) windowspecdefinition(four#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS count(1) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL], [four#x] + +- Project [four#x] + +- SubqueryAlias s + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- Filter false + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1) +-- !query analysis +CreateViewCommand `int4_tbl`, select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1), false, false, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias int4_tbl + +- LocalRelation [f1#x] + + +-- !query +SELECT SUM(COUNT(f1)) OVER () FROM int4_tbl WHERE f1=42 +-- !query analysis +Project [sum(count(f1)) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] ++- Project [_w0#xL, sum(count(f1)) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, sum(count(f1)) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] + +- Window [sum(_w0#xL) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS sum(count(f1)) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] + +- Aggregate [count(f1#x) AS _w0#xL] + +- Filter (f1#x = 42) + +- SubqueryAlias int4_tbl + +- View (`int4_tbl`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias int4_tbl + +- LocalRelation [f1#x] + + +-- !query +select ten, + sum(unique1) + sum(unique2) as res, + rank() over (order by sum(unique1) + sum(unique2)) as rank +from tenk1 +group by ten order by ten +-- !query analysis +Sort [ten#x ASC NULLS FIRST], true ++- Project [ten#x, res#xL, rank#x] + +- Project [ten#x, res#xL, _w0#xL, rank#x, rank#x] + +- Window [rank(_w0#xL) windowspecdefinition(_w0#xL ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#x], [_w0#xL ASC NULLS FIRST] + +- Aggregate [ten#x], [ten#x, (sum(unique1#x) + sum(unique2#x)) AS res#xL, (sum(unique1#x) + sum(unique2#x)) AS _w0#xL] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT four, ten, +sum(ten) over (partition by four order by ten), +last(ten) over (partition by four order by ten) +FROM (select distinct ten, four from tenk1) ss +-- !query analysis +Project [four#x, ten#x, sum(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, last(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [four#x, ten#x, sum(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, last(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, sum(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, last(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [sum(ten#x) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS sum(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, last(ten#x, false) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [four#x], [ten#x ASC NULLS FIRST] + +- Project [four#x, ten#x] + +- SubqueryAlias ss + +- Distinct + +- Project [ten#x, four#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and current row), +last(ten) over (partition by four order by ten range between unbounded preceding and current row) +FROM (select distinct ten, four from tenk1) ss +-- !query analysis +Project [four#x, ten#x, sum(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, last(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [four#x, ten#x, sum(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, last(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, sum(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, last(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [sum(ten#x) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS sum(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, last(ten#x, false) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [four#x], [ten#x ASC NULLS FIRST] + +- Project [four#x, ten#x] + +- SubqueryAlias ss + +- Distinct + +- Project [ten#x, four#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and unbounded following), +last(ten) over (partition by four order by ten range between unbounded preceding and unbounded following) +FROM (select distinct ten, four from tenk1) ss +-- !query analysis +Project [four#x, ten#x, sum(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, last(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] ++- Project [four#x, ten#x, sum(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, last(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, sum(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, last(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Window [sum(ten#x) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), unboundedfollowing$())) AS sum(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, last(ten#x, false) windowspecdefinition(four#x, ten#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), unboundedfollowing$())) AS last(ten) OVER (PARTITION BY four ORDER BY ten ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x], [four#x], [ten#x ASC NULLS FIRST] + +- Project [four#x, ten#x] + +- SubqueryAlias ss + +- Distinct + +- Project [ten#x, four#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT sum(unique1) over (order by four range between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query analysis +Project [sum(unique1) OVER (ORDER BY four ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL, unique1#x, four#x] ++- Project [unique1#x, four#x, sum(unique1) OVER (ORDER BY four ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL, sum(unique1) OVER (ORDER BY four ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL] + +- Window [sum(unique1#x) windowspecdefinition(four#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), unboundedfollowing$())) AS sum(unique1) OVER (ORDER BY four ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL], [four#x ASC NULLS FIRST] + +- Project [unique1#x, four#x] + +- Filter (unique1#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT sum(unique1) over (rows between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query analysis +Project [sum(unique1) OVER (ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL, unique1#x, four#x] ++- Project [unique1#x, four#x, sum(unique1) OVER (ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL, sum(unique1) OVER (ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL] + +- Window [sum(unique1#x) windowspecdefinition(specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS sum(unique1) OVER (ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL] + +- Project [unique1#x, four#x] + +- Filter (unique1#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT sum(unique1) over (rows between 2 preceding and 2 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query analysis +Project [sum(unique1) OVER (ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING)#xL, unique1#x, four#x] ++- Project [unique1#x, four#x, sum(unique1) OVER (ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING)#xL, sum(unique1) OVER (ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING)#xL] + +- Window [sum(unique1#x) windowspecdefinition(specifiedwindowframe(RowFrame, -2, 2)) AS sum(unique1) OVER (ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING)#xL] + +- Project [unique1#x, four#x] + +- Filter (unique1#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT sum(unique1) over (rows between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query analysis +Project [sum(unique1) OVER (ROWS BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL, unique1#x, four#x] ++- Project [unique1#x, four#x, sum(unique1) OVER (ROWS BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL, sum(unique1) OVER (ROWS BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL] + +- Window [sum(unique1#x) windowspecdefinition(specifiedwindowframe(RowFrame, -2, -1)) AS sum(unique1) OVER (ROWS BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL] + +- Project [unique1#x, four#x] + +- Filter (unique1#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT sum(unique1) over (rows between 1 following and 3 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query analysis +Project [sum(unique1) OVER (ROWS BETWEEN 1 FOLLOWING AND 3 FOLLOWING)#xL, unique1#x, four#x] ++- Project [unique1#x, four#x, sum(unique1) OVER (ROWS BETWEEN 1 FOLLOWING AND 3 FOLLOWING)#xL, sum(unique1) OVER (ROWS BETWEEN 1 FOLLOWING AND 3 FOLLOWING)#xL] + +- Window [sum(unique1#x) windowspecdefinition(specifiedwindowframe(RowFrame, 1, 3)) AS sum(unique1) OVER (ROWS BETWEEN 1 FOLLOWING AND 3 FOLLOWING)#xL] + +- Project [unique1#x, four#x] + +- Filter (unique1#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT sum(unique1) over (rows between unbounded preceding and 1 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query analysis +Project [sum(unique1) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)#xL, unique1#x, four#x] ++- Project [unique1#x, four#x, sum(unique1) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)#xL, sum(unique1) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)#xL] + +- Window [sum(unique1#x) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), 1)) AS sum(unique1) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)#xL] + +- Project [unique1#x, four#x] + +- Filter (unique1#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +CREATE TEMP VIEW v_window AS +SELECT i.id, sum(i.id) over (order by i.id rows between 1 preceding and 1 following) as sum_rows +FROM range(1, 11) i +-- !query analysis +CreateViewCommand `v_window`, SELECT i.id, sum(i.id) over (order by i.id rows between 1 preceding and 1 following) as sum_rows +FROM range(1, 11) i, false, false, LocalTempView, true + +- Project [id#xL, sum_rows#xL] + +- Project [id#xL, sum_rows#xL, sum_rows#xL] + +- Window [sum(id#xL) windowspecdefinition(id#xL ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, 1)) AS sum_rows#xL], [id#xL ASC NULLS FIRST] + +- Project [id#xL] + +- SubqueryAlias i + +- Range (1, 11, step=1, splits=None) + + +-- !query +SELECT * FROM v_window +-- !query analysis +Project [id#xL, sum_rows#xL] ++- SubqueryAlias v_window + +- View (`v_window`, [id#xL,sum_rows#xL]) + +- Project [cast(id#xL as bigint) AS id#xL, cast(sum_rows#xL as bigint) AS sum_rows#xL] + +- Project [id#xL, sum_rows#xL] + +- Project [id#xL, sum_rows#xL, sum_rows#xL] + +- Window [sum(id#xL) windowspecdefinition(id#xL ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, 1)) AS sum_rows#xL], [id#xL ASC NULLS FIRST] + +- Project [id#xL] + +- SubqueryAlias i + +- Range (1, 11, step=1, splits=None) + + +-- !query +DROP VIEW v_window +-- !query analysis +DropTempViewCommand v_window + + +-- !query +DROP VIEW tenk2 +-- !query analysis +DropTempViewCommand tenk2 + + +-- !query +DROP VIEW int4_tbl +-- !query analysis +DropTempViewCommand int4_tbl diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part2.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part2.sql.out new file mode 100644 index 0000000000000..e8d8c0692769a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part2.sql.out @@ -0,0 +1,476 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE empsalary ( + depname string, + empno integer, + salary int, + enroll_date date +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`empsalary`, false + + +-- !query +INSERT INTO empsalary VALUES + ('develop', 10, 5200, date '2007-08-01'), + ('sales', 1, 5000, date '2006-10-01'), + ('personnel', 5, 3500, date '2007-12-10'), + ('sales', 4, 4800, date '2007-08-08'), + ('personnel', 2, 3900, date '2006-12-23'), + ('develop', 7, 4200, date '2008-01-01'), + ('develop', 9, 4500, date '2008-01-01'), + ('sales', 3, 4800, date '2007-08-01'), + ('develop', 8, 6000, date '2006-10-01'), + ('develop', 11, 5200, date '2007-08-15') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/empsalary, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/empsalary], Append, `spark_catalog`.`default`.`empsalary`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/empsalary), [depname, empno, salary, enroll_date] ++- Project [cast(col1#x as string) AS depname#x, cast(col2#x as int) AS empno#x, cast(col3#x as int) AS salary#x, cast(col4#x as date) AS enroll_date#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query analysis +Project [sum(unique1) OVER (ORDER BY four ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL, unique1#x, four#x] ++- Project [unique1#x, four#x, sum(unique1) OVER (ORDER BY four ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL, sum(unique1) OVER (ORDER BY four ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL] + +- Window [sum(unique1#x) windowspecdefinition(four#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -2, -1)) AS sum(unique1) OVER (ORDER BY four ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL], [four#x ASC NULLS FIRST] + +- Project [unique1#x, four#x] + +- Filter (unique1#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT sum(unique1) over (order by four desc range between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query analysis +Project [sum(unique1) OVER (ORDER BY four DESC NULLS LAST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL, unique1#x, four#x] ++- Project [unique1#x, four#x, sum(unique1) OVER (ORDER BY four DESC NULLS LAST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL, sum(unique1) OVER (ORDER BY four DESC NULLS LAST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL] + +- Window [sum(unique1#x) windowspecdefinition(four#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, -2, -1)) AS sum(unique1) OVER (ORDER BY four DESC NULLS LAST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL], [four#x DESC NULLS LAST] + +- Project [unique1#x, four#x] + +- Filter (unique1#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT sum(unique1) over (partition by four order by unique1 range between 5 preceding and 6 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query analysis +Project [sum(unique1) OVER (PARTITION BY four ORDER BY unique1 ASC NULLS FIRST RANGE BETWEEN 5 PRECEDING AND 6 FOLLOWING)#xL, unique1#x, four#x] ++- Project [unique1#x, four#x, sum(unique1) OVER (PARTITION BY four ORDER BY unique1 ASC NULLS FIRST RANGE BETWEEN 5 PRECEDING AND 6 FOLLOWING)#xL, sum(unique1) OVER (PARTITION BY four ORDER BY unique1 ASC NULLS FIRST RANGE BETWEEN 5 PRECEDING AND 6 FOLLOWING)#xL] + +- Window [sum(unique1#x) windowspecdefinition(four#x, unique1#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -5, 6)) AS sum(unique1) OVER (PARTITION BY four ORDER BY unique1 ASC NULLS FIRST RANGE BETWEEN 5 PRECEDING AND 6 FOLLOWING)#xL], [four#x], [unique1#x ASC NULLS FIRST] + +- Project [unique1#x, four#x] + +- Filter (unique1#x < 10) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id asc nulls first range between 2 preceding and 2 following) +-- !query analysis +Project [id#xL, y#xL, first(y) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL] ++- Project [id#xL, y#xL, first(y) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, first(y) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL] + +- Window [first(y#xL, false) windowspecdefinition(id#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-2 as bigint), cast(2 as bigint))) AS first(y) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y#xL, false) windowspecdefinition(id#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-2 as bigint), cast(2 as bigint))) AS last(y) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL], [id#xL ASC NULLS FIRST] + +- Project [id#xL, y#xL] + +- SubqueryAlias ss + +- Union false, false + :- Union false, false + : :- Project [id#xL, id#xL AS y#xL] + : : +- SubqueryAlias x + : : +- Range (1, 6, step=1, splits=None) + : +- Project [cast(NULL#x as bigint) AS NULL#xL, cast(42#x as bigint) AS 42#xL] + : +- Project [null AS NULL#x, 42 AS 42#x] + : +- OneRowRelation + +- Project [cast(NULL#x as bigint) AS NULL#xL, cast(43#x as bigint) AS 43#xL] + +- Project [null AS NULL#x, 43 AS 43#x] + +- OneRowRelation + + +-- !query +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id asc nulls last range between 2 preceding and 2 following) +-- !query analysis +Project [id#xL, y#xL, first(y) OVER (ORDER BY id ASC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y) OVER (ORDER BY id ASC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL] ++- Project [id#xL, y#xL, first(y) OVER (ORDER BY id ASC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y) OVER (ORDER BY id ASC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, first(y) OVER (ORDER BY id ASC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y) OVER (ORDER BY id ASC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL] + +- Window [first(y#xL, false) windowspecdefinition(id#xL ASC NULLS LAST, specifiedwindowframe(RangeFrame, cast(-2 as bigint), cast(2 as bigint))) AS first(y) OVER (ORDER BY id ASC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y#xL, false) windowspecdefinition(id#xL ASC NULLS LAST, specifiedwindowframe(RangeFrame, cast(-2 as bigint), cast(2 as bigint))) AS last(y) OVER (ORDER BY id ASC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL], [id#xL ASC NULLS LAST] + +- Project [id#xL, y#xL] + +- SubqueryAlias ss + +- Union false, false + :- Union false, false + : :- Project [id#xL, id#xL AS y#xL] + : : +- SubqueryAlias x + : : +- Range (1, 6, step=1, splits=None) + : +- Project [cast(NULL#x as bigint) AS NULL#xL, cast(42#x as bigint) AS 42#xL] + : +- Project [null AS NULL#x, 42 AS 42#x] + : +- OneRowRelation + +- Project [cast(NULL#x as bigint) AS NULL#xL, cast(43#x as bigint) AS 43#xL] + +- Project [null AS NULL#x, 43 AS 43#x] + +- OneRowRelation + + +-- !query +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id desc nulls first range between 2 preceding and 2 following) +-- !query analysis +Project [id#xL, y#xL, first(y) OVER (ORDER BY id DESC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y) OVER (ORDER BY id DESC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL] ++- Project [id#xL, y#xL, first(y) OVER (ORDER BY id DESC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y) OVER (ORDER BY id DESC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, first(y) OVER (ORDER BY id DESC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y) OVER (ORDER BY id DESC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL] + +- Window [first(y#xL, false) windowspecdefinition(id#xL DESC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-2 as bigint), cast(2 as bigint))) AS first(y) OVER (ORDER BY id DESC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y#xL, false) windowspecdefinition(id#xL DESC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-2 as bigint), cast(2 as bigint))) AS last(y) OVER (ORDER BY id DESC NULLS FIRST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL], [id#xL DESC NULLS FIRST] + +- Project [id#xL, y#xL] + +- SubqueryAlias ss + +- Union false, false + :- Union false, false + : :- Project [id#xL, id#xL AS y#xL] + : : +- SubqueryAlias x + : : +- Range (1, 6, step=1, splits=None) + : +- Project [cast(NULL#x as bigint) AS NULL#xL, cast(42#x as bigint) AS 42#xL] + : +- Project [null AS NULL#x, 42 AS 42#x] + : +- OneRowRelation + +- Project [cast(NULL#x as bigint) AS NULL#xL, cast(43#x as bigint) AS 43#xL] + +- Project [null AS NULL#x, 43 AS 43#x] + +- OneRowRelation + + +-- !query +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id desc nulls last range between 2 preceding and 2 following) +-- !query analysis +Project [id#xL, y#xL, first(y) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL] ++- Project [id#xL, y#xL, first(y) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, first(y) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL] + +- Window [first(y#xL, false) windowspecdefinition(id#xL DESC NULLS LAST, specifiedwindowframe(RangeFrame, cast(-2 as bigint), cast(2 as bigint))) AS first(y) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL, last(y#xL, false) windowspecdefinition(id#xL DESC NULLS LAST, specifiedwindowframe(RangeFrame, cast(-2 as bigint), cast(2 as bigint))) AS last(y) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN (- 2) FOLLOWING AND 2 FOLLOWING)#xL], [id#xL DESC NULLS LAST] + +- Project [id#xL, y#xL] + +- SubqueryAlias ss + +- Union false, false + :- Union false, false + : :- Project [id#xL, id#xL AS y#xL] + : : +- SubqueryAlias x + : : +- Range (1, 6, step=1, splits=None) + : +- Project [cast(NULL#x as bigint) AS NULL#xL, cast(42#x as bigint) AS 42#xL] + : +- Project [null AS NULL#x, 42 AS 42#x] + : +- OneRowRelation + +- Project [cast(NULL#x as bigint) AS NULL#xL, cast(43#x as bigint) AS 43#xL] + +- Project [null AS NULL#x, 43 AS 43#x] + +- OneRowRelation + + +-- !query +select x.id, last(x.id) over (order by x.id range between current row and 2147450884 following) +from range(32764, 32767) x +-- !query analysis +Project [id#xL, last(id) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2147450884 FOLLOWING)#xL] ++- Project [id#xL, last(id) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2147450884 FOLLOWING)#xL, last(id) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2147450884 FOLLOWING)#xL] + +- Window [last(id#xL, false) windowspecdefinition(id#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), cast(2147450884 as bigint))) AS last(id) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2147450884 FOLLOWING)#xL], [id#xL ASC NULLS FIRST] + +- Project [id#xL] + +- SubqueryAlias x + +- Range (32764, 32767, step=1, splits=None) + + +-- !query +select x.id, last(x.id) over (order by x.id desc range between current row and 2147450885 following) +from range(-32766, -32765) x +-- !query analysis +Project [id#xL, last(id) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 2147450885 FOLLOWING)#xL] ++- Project [id#xL, last(id) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 2147450885 FOLLOWING)#xL, last(id) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 2147450885 FOLLOWING)#xL] + +- Window [last(id#xL, false) windowspecdefinition(id#xL DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), cast(2147450885 as bigint))) AS last(id) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 2147450885 FOLLOWING)#xL], [id#xL DESC NULLS LAST] + +- Project [id#xL] + +- SubqueryAlias x + +- Range (-32766, -32765, step=1, splits=None) + + +-- !query +select x.id, last(x.id) over (order by x.id range between current row and 4 following) +from range(2147483644, 2147483647) x +-- !query analysis +Project [id#xL, last(id) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 4 FOLLOWING)#xL] ++- Project [id#xL, last(id) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 4 FOLLOWING)#xL, last(id) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 4 FOLLOWING)#xL] + +- Window [last(id#xL, false) windowspecdefinition(id#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), cast(4 as bigint))) AS last(id) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 4 FOLLOWING)#xL], [id#xL ASC NULLS FIRST] + +- Project [id#xL] + +- SubqueryAlias x + +- Range (2147483644, 2147483647, step=1, splits=None) + + +-- !query +select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) +from range(-2147483646, -2147483645) x +-- !query analysis +Project [id#xL, last(id) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 5 FOLLOWING)#xL] ++- Project [id#xL, last(id) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 5 FOLLOWING)#xL, last(id) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 5 FOLLOWING)#xL] + +- Window [last(id#xL, false) windowspecdefinition(id#xL DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), cast(5 as bigint))) AS last(id) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 5 FOLLOWING)#xL], [id#xL DESC NULLS LAST] + +- Project [id#xL] + +- SubqueryAlias x + +- Range (-2147483646, -2147483645, step=1, splits=None) + + +-- !query +select x.id, last(x.id) over (order by x.id range between current row and 4 following) +from range(9223372036854775804, 9223372036854775807) x +-- !query analysis +Project [id#xL, last(id) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 4 FOLLOWING)#xL] ++- Project [id#xL, last(id) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 4 FOLLOWING)#xL, last(id) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 4 FOLLOWING)#xL] + +- Window [last(id#xL, false) windowspecdefinition(id#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), cast(4 as bigint))) AS last(id) OVER (ORDER BY id ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 4 FOLLOWING)#xL], [id#xL ASC NULLS FIRST] + +- Project [id#xL] + +- SubqueryAlias x + +- Range (9223372036854775804, 9223372036854775807, step=1, splits=None) + + +-- !query +select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) +from range(-9223372036854775806, -9223372036854775805) x +-- !query analysis +Project [id#xL, last(id) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 5 FOLLOWING)#xL] ++- Project [id#xL, last(id) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 5 FOLLOWING)#xL, last(id) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 5 FOLLOWING)#xL] + +- Window [last(id#xL, false) windowspecdefinition(id#xL DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), cast(5 as bigint))) AS last(id) OVER (ORDER BY id DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 5 FOLLOWING)#xL], [id#xL DESC NULLS LAST] + +- Project [id#xL] + +- SubqueryAlias x + +- Range (-9223372036854775806, -9223372036854775805, step=1, splits=None) + + +-- !query +create table numerics ( + id int, + f_float4 float, + f_float8 float, + f_numeric int +) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`numerics`, false + + +-- !query +insert into numerics values +(1, -3, -3, -3), +(2, -1, -1, -1), +(3, 0, 0, 0), +(4, 1.1, 1.1, 1.1), +(5, 1.12, 1.12, 1.12), +(6, 2, 2, 2), +(7, 100, 100, 100) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/numerics, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/numerics], Append, `spark_catalog`.`default`.`numerics`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/numerics), [id, f_float4, f_float8, f_numeric] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as float) AS f_float4#x, cast(col3#x as float) AS f_float8#x, cast(col4#x as int) AS f_numeric#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1 preceding and 1 following) +-- !query analysis +Project [id#x, f_float4#x, first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x, last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x] ++- Project [id#x, f_float4#x, first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x, last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x, first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x, last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x] + +- Window [first(id#x, false) windowspecdefinition(f_float4#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as float), cast(1 as float))) AS first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x, last(id#x, false) windowspecdefinition(f_float4#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as float), cast(1 as float))) AS last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x], [f_float4#x ASC NULLS FIRST] + +- Project [id#x, f_float4#x] + +- SubqueryAlias spark_catalog.default.numerics + +- Relation spark_catalog.default.numerics[id#x,f_float4#x,f_float8#x,f_numeric#x] parquet + + +-- !query +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1 preceding and 1.1 following) +-- !query analysis +Project [id#x, f_float4#x, first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x, last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x] ++- Project [id#x, f_float4#x, first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x, last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x, first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x, last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x] + +- Window [first(id#x, false) windowspecdefinition(f_float4#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as float), cast(1.1 as float))) AS first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x, last(id#x, false) windowspecdefinition(f_float4#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as float), cast(1.1 as float))) AS last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x], [f_float4#x ASC NULLS FIRST] + +- Project [id#x, f_float4#x] + +- SubqueryAlias spark_catalog.default.numerics + +- Relation spark_catalog.default.numerics[id#x,f_float4#x,f_float8#x,f_numeric#x] parquet + + +-- !query +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 'inf' preceding and 'inf' following) +-- !query analysis +Project [id#x, f_float4#x, first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x, last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x] ++- Project [id#x, f_float4#x, first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x, last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x, first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x, last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x] + +- Window [first(id#x, false) windowspecdefinition(f_float4#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-cast(inf as double) as float), cast(inf as float))) AS first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x, last(id#x, false) windowspecdefinition(f_float4#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-cast(inf as double) as float), cast(inf as float))) AS last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x], [f_float4#x ASC NULLS FIRST] + +- Project [id#x, f_float4#x] + +- SubqueryAlias spark_catalog.default.numerics + +- Relation spark_catalog.default.numerics[id#x,f_float4#x,f_float8#x,f_numeric#x] parquet + + +-- !query +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1.1 preceding and 'NaN' following) +-- !query analysis +Project [id#x, f_float4#x, first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x, last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x] ++- Project [id#x, f_float4#x, first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x, last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x, first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x, last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x] + +- Window [first(id#x, false) windowspecdefinition(f_float4#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1.1 as float), cast(NaN as float))) AS first(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x, last(id#x, false) windowspecdefinition(f_float4#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1.1 as float), cast(NaN as float))) AS last(id) OVER (ORDER BY f_float4 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x], [f_float4#x ASC NULLS FIRST] + +- Project [id#x, f_float4#x] + +- SubqueryAlias spark_catalog.default.numerics + +- Relation spark_catalog.default.numerics[id#x,f_float4#x,f_float8#x,f_numeric#x] parquet + + +-- !query +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1 preceding and 1 following) +-- !query analysis +Project [id#x, f_float8#x, first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x, last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x] ++- Project [id#x, f_float8#x, first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x, last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x, first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x, last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x] + +- Window [first(id#x, false) windowspecdefinition(f_float8#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as float), cast(1 as float))) AS first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x, last(id#x, false) windowspecdefinition(f_float8#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as float), cast(1 as float))) AS last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#x], [f_float8#x ASC NULLS FIRST] + +- Project [id#x, f_float8#x] + +- SubqueryAlias spark_catalog.default.numerics + +- Relation spark_catalog.default.numerics[id#x,f_float4#x,f_float8#x,f_numeric#x] parquet + + +-- !query +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1 preceding and 1.1 following) +-- !query analysis +Project [id#x, f_float8#x, first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x, last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x] ++- Project [id#x, f_float8#x, first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x, last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x, first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x, last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x] + +- Window [first(id#x, false) windowspecdefinition(f_float8#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as float), cast(1.1 as float))) AS first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x, last(id#x, false) windowspecdefinition(f_float8#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as float), cast(1.1 as float))) AS last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1.1 FOLLOWING)#x], [f_float8#x ASC NULLS FIRST] + +- Project [id#x, f_float8#x] + +- SubqueryAlias spark_catalog.default.numerics + +- Relation spark_catalog.default.numerics[id#x,f_float4#x,f_float8#x,f_numeric#x] parquet + + +-- !query +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 'inf' preceding and 'inf' following) +-- !query analysis +Project [id#x, f_float8#x, first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x, last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x] ++- Project [id#x, f_float8#x, first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x, last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x, first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x, last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x] + +- Window [first(id#x, false) windowspecdefinition(f_float8#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-cast(inf as double) as float), cast(inf as float))) AS first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x, last(id#x, false) windowspecdefinition(f_float8#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-cast(inf as double) as float), cast(inf as float))) AS last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- inf) FOLLOWING AND inf FOLLOWING)#x], [f_float8#x ASC NULLS FIRST] + +- Project [id#x, f_float8#x] + +- SubqueryAlias spark_catalog.default.numerics + +- Relation spark_catalog.default.numerics[id#x,f_float4#x,f_float8#x,f_numeric#x] parquet + + +-- !query +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1.1 preceding and 'NaN' following) +-- !query analysis +Project [id#x, f_float8#x, first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x, last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x] ++- Project [id#x, f_float8#x, first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x, last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x, first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x, last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x] + +- Window [first(id#x, false) windowspecdefinition(f_float8#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1.1 as float), cast(NaN as float))) AS first(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x, last(id#x, false) windowspecdefinition(f_float8#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1.1 as float), cast(NaN as float))) AS last(id) OVER (ORDER BY f_float8 ASC NULLS FIRST RANGE BETWEEN (- 1.1) FOLLOWING AND NaN FOLLOWING)#x], [f_float8#x ASC NULLS FIRST] + +- Project [id#x, f_float8#x] + +- SubqueryAlias spark_catalog.default.numerics + +- Relation spark_catalog.default.numerics[id#x,f_float4#x,f_float8#x,f_numeric#x] parquet + + +-- !query +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1 following) +-- !query analysis +Project [id#x, f_numeric#x, first(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#x, last(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#x] ++- Project [id#x, f_numeric#x, first(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#x, last(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#x, first(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#x, last(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#x] + +- Window [first(id#x, false) windowspecdefinition(f_numeric#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -1, 1)) AS first(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#x, last(id#x, false) windowspecdefinition(f_numeric#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -1, 1)) AS last(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#x], [f_numeric#x ASC NULLS FIRST] + +- Project [id#x, f_numeric#x] + +- SubqueryAlias spark_catalog.default.numerics + +- Relation spark_catalog.default.numerics[id#x,f_float4#x,f_float8#x,f_numeric#x] parquet + + +-- !query +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1.1 following) +-- !query analysis +Project [id#x, f_numeric#x, first(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x, last(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x] ++- Project [id#x, f_numeric#x, first(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x, last(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x, first(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x, last(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x] + +- Window [first(id#x, false) windowspecdefinition(f_numeric#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -1, cast(1.1 as int))) AS first(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x, last(id#x, false) windowspecdefinition(f_numeric#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -1, cast(1.1 as int))) AS last(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x], [f_numeric#x ASC NULLS FIRST] + +- Project [id#x, f_numeric#x] + +- SubqueryAlias spark_catalog.default.numerics + +- Relation spark_catalog.default.numerics[id#x,f_float4#x,f_float8#x,f_numeric#x] parquet + + +-- !query +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1.1 following) +-- !query analysis +Project [id#x, f_numeric#x, first(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x, last(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x] ++- Project [id#x, f_numeric#x, first(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x, last(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x, first(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x, last(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x] + +- Window [first(id#x, false) windowspecdefinition(f_numeric#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -1, cast(1.1 as int))) AS first(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x, last(id#x, false) windowspecdefinition(f_numeric#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -1, cast(1.1 as int))) AS last(id) OVER (ORDER BY f_numeric ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1.1 FOLLOWING)#x], [f_numeric#x ASC NULLS FIRST] + +- Project [id#x, f_numeric#x] + +- SubqueryAlias spark_catalog.default.numerics + +- Relation spark_catalog.default.numerics[id#x,f_float4#x,f_float8#x,f_numeric#x] parquet + + +-- !query +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1.1 preceding and 'NaN' following) +-- !query analysis +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'NaN'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 83, + "stopIndex" : 163, + "fragment" : "(order by f_numeric range between\n 1.1 preceding and 'NaN' following)" + } ] +} + + +-- !query +drop table empsalary +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`empsalary`, false, false, false + + +-- !query +drop table numerics +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`numerics`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out new file mode 100644 index 0000000000000..52b58a4a296a9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out @@ -0,0 +1,488 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 +-- !query analysis +CreateViewCommand `tenk2`, SELECT * FROM tenk1, false, false, LocalTempView, true + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +CREATE TABLE empsalary ( + depname string, + empno integer, + salary int, + enroll_date date +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`empsalary`, false + + +-- !query +INSERT INTO empsalary VALUES + ('develop', 10, 5200, date '2007-08-01'), + ('sales', 1, 5000, date '2006-10-01'), + ('personnel', 5, 3500, date '2007-12-10'), + ('sales', 4, 4800, date '2007-08-08'), + ('personnel', 2, 3900, date '2006-12-23'), + ('develop', 7, 4200, date '2008-01-01'), + ('develop', 9, 4500, date '2008-01-01'), + ('sales', 3, 4800, date '2007-08-01'), + ('develop', 8, 6000, date '2006-10-01'), + ('develop', 11, 5200, date '2007-08-15') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/empsalary, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/empsalary], Append, `spark_catalog`.`default`.`empsalary`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/empsalary), [depname, empno, salary, enroll_date] ++- Project [cast(col1#x as string) AS depname#x, cast(col2#x as int) AS empno#x, cast(col3#x as int) AS salary#x, cast(col4#x as date) AS enroll_date#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +create table datetimes ( + id int, + f_time timestamp, + f_timetz timestamp, + f_interval timestamp, + f_timestamptz timestamp, + f_timestamp timestamp +) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`datetimes`, false + + +-- !query +insert into datetimes values +(1, timestamp '11:00', cast ('11:00 BST' as timestamp), cast ('1 year' as timestamp), cast ('2000-10-19 10:23:54+01' as timestamp), timestamp '2000-10-19 10:23:54'), +(2, timestamp '12:00', cast ('12:00 BST' as timestamp), cast ('2 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'), +(3, timestamp '13:00', cast ('13:00 BST' as timestamp), cast ('3 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'), +(4, timestamp '14:00', cast ('14:00 BST' as timestamp), cast ('4 years' as timestamp), cast ('2002-10-19 10:23:54+01' as timestamp), timestamp '2002-10-19 10:23:54'), +(5, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2003-10-19 10:23:54+01' as timestamp), timestamp '2003-10-19 10:23:54'), +(6, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2004-10-19 10:23:54+01' as timestamp), timestamp '2004-10-19 10:23:54'), +(7, timestamp '17:00', cast ('17:00 BST' as timestamp), cast ('7 years' as timestamp), cast ('2005-10-19 10:23:54+01' as timestamp), timestamp '2005-10-19 10:23:54'), +(8, timestamp '18:00', cast ('18:00 BST' as timestamp), cast ('8 years' as timestamp), cast ('2006-10-19 10:23:54+01' as timestamp), timestamp '2006-10-19 10:23:54'), +(9, timestamp '19:00', cast ('19:00 BST' as timestamp), cast ('9 years' as timestamp), cast ('2007-10-19 10:23:54+01' as timestamp), timestamp '2007-10-19 10:23:54'), +(10, timestamp '20:00', cast ('20:00 BST' as timestamp), cast ('10 years' as timestamp), cast ('2008-10-19 10:23:54+01' as timestamp), timestamp '2008-10-19 10:23:54') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2331", + "messageParameters" : { + "msg" : "[CAST_INVALID_INPUT] The value '11:00 BST' of the type \"STRING\" cannot be cast to \"TIMESTAMP\" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.\n== SQL(line 2, position 24) ==\n(1, timestamp '11:00', cast ('11:00 BST' as timestamp), cast ('1 year' as timestamp), ...\n ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^\n", + "sqlExpr" : "CAST('11:00 BST' AS TIMESTAMP)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 1698, + "fragment" : "values\n(1, timestamp '11:00', cast ('11:00 BST' as timestamp), cast ('1 year' as timestamp), cast ('2000-10-19 10:23:54+01' as timestamp), timestamp '2000-10-19 10:23:54'),\n(2, timestamp '12:00', cast ('12:00 BST' as timestamp), cast ('2 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'),\n(3, timestamp '13:00', cast ('13:00 BST' as timestamp), cast ('3 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'),\n(4, timestamp '14:00', cast ('14:00 BST' as timestamp), cast ('4 years' as timestamp), cast ('2002-10-19 10:23:54+01' as timestamp), timestamp '2002-10-19 10:23:54'),\n(5, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2003-10-19 10:23:54+01' as timestamp), timestamp '2003-10-19 10:23:54'),\n(6, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2004-10-19 10:23:54+01' as timestamp), timestamp '2004-10-19 10:23:54'),\n(7, timestamp '17:00', cast ('17:00 BST' as timestamp), cast ('7 years' as timestamp), cast ('2005-10-19 10:23:54+01' as timestamp), timestamp '2005-10-19 10:23:54'),\n(8, timestamp '18:00', cast ('18:00 BST' as timestamp), cast ('8 years' as timestamp), cast ('2006-10-19 10:23:54+01' as timestamp), timestamp '2006-10-19 10:23:54'),\n(9, timestamp '19:00', cast ('19:00 BST' as timestamp), cast ('9 years' as timestamp), cast ('2007-10-19 10:23:54+01' as timestamp), timestamp '2007-10-19 10:23:54'),\n(10, timestamp '20:00', cast ('20:00 BST' as timestamp), cast ('10 years' as timestamp), cast ('2008-10-19 10:23:54+01' as timestamp), timestamp '2008-10-19 10:23:54')" + } ] +} + + +-- !query +WITH cte (x) AS ( + SELECT * FROM range(1, 36, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte +: +- Project [id#xL AS x#xL] +: +- Project [id#xL] +: +- Range (1, 36, step=2, splits=None) ++- Project [x#xL, sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL] + +- Project [x#xL, sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL, sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL] + +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, 1)) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] + +- Project [x#xL] + +- SubqueryAlias cte + +- CTERelationRef xxxx, true, [x#xL] + + +-- !query +WITH cte (x) AS ( + SELECT * FROM range(1, 36, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte +: +- Project [id#xL AS x#xL] +: +- Project [id#xL] +: +- Range (1, 36, step=2, splits=None) ++- Project [x#xL, sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL] + +- Project [x#xL, sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL, sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL] + +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as bigint), cast(1 as bigint))) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] + +- Project [x#xL] + +- SubqueryAlias cte + +- CTERelationRef xxxx, true, [x#xL] + + +-- !query +WITH cte (x) AS ( + select 1 union all select 1 union all select 1 union all + SELECT * FROM range(5, 50, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte +: +- Project [1#xL AS x#xL] +: +- Union false, false +: :- Project [cast(1#x as bigint) AS 1#xL] +: : +- Union false, false +: : :- Union false, false +: : : :- Project [1 AS 1#x] +: : : : +- OneRowRelation +: : : +- Project [1 AS 1#x] +: : : +- OneRowRelation +: : +- Project [1 AS 1#x] +: : +- OneRowRelation +: +- Project [id#xL] +: +- Range (5, 50, step=2, splits=None) ++- Project [x#xL, sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL] + +- Project [x#xL, sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL, sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL] + +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, 1)) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] + +- Project [x#xL] + +- SubqueryAlias cte + +- CTERelationRef xxxx, true, [x#xL] + + +-- !query +WITH cte (x) AS ( + select 1 union all select 1 union all select 1 union all + SELECT * FROM range(5, 50, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte +: +- Project [1#xL AS x#xL] +: +- Union false, false +: :- Project [cast(1#x as bigint) AS 1#xL] +: : +- Union false, false +: : :- Union false, false +: : : :- Project [1 AS 1#x] +: : : : +- OneRowRelation +: : : +- Project [1 AS 1#x] +: : : +- OneRowRelation +: : +- Project [1 AS 1#x] +: : +- OneRowRelation +: +- Project [id#xL] +: +- Range (5, 50, step=2, splits=None) ++- Project [x#xL, sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL] + +- Project [x#xL, sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL, sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL] + +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as bigint), cast(1 as bigint))) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] + +- Project [x#xL] + +- SubqueryAlias cte + +- CTERelationRef xxxx, true, [x#xL] + + +-- !query +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 UNION ALL SELECT * FROM tenk2)s LIMIT 0 +-- !query analysis +GlobalLimit 0 ++- LocalLimit 0 + +- Project [count(1) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] + +- Project [four#x, count(1) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, count(1) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL] + +- Window [count(1) windowspecdefinition(four#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS count(1) OVER (PARTITION BY four ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL], [four#x] + +- Project [four#x] + +- SubqueryAlias s + +- Union false, false + :- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias tenk2 + +- View (`tenk2`, [unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x]) + +- Project [cast(unique1#x as int) AS unique1#x, cast(unique2#x as int) AS unique2#x, cast(two#x as int) AS two#x, cast(four#x as int) AS four#x, cast(ten#x as int) AS ten#x, cast(twenty#x as int) AS twenty#x, cast(hundred#x as int) AS hundred#x, cast(thousand#x as int) AS thousand#x, cast(twothousand#x as int) AS twothousand#x, cast(fivethous#x as int) AS fivethous#x, cast(tenthous#x as int) AS tenthous#x, cast(odd#x as int) AS odd#x, cast(even#x as int) AS even#x, cast(stringu1#x as string) AS stringu1#x, cast(stringu2#x as string) AS stringu2#x, cast(string4#x as string) AS string4#x] + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +create table t1 (f1 int, f2 int) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false + + +-- !query +insert into t1 values (1,1),(1,2),(2,2) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [f1, f2] ++- Project [cast(col1#x as int) AS f1#x, cast(col2#x as int) AS f2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select f1, sum(f1) over (partition by f1 + range between 1 preceding and 1 following) +from t1 where f1 = f2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"(PARTITION BY f1 RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 108, + "fragment" : "(partition by f1\n range between 1 preceding and 1 following)" + } ] +} + + +-- !query +select f1, sum(f1) over (partition by f1 order by f2 +range between 1 preceding and 1 following) +from t1 where f1 = f2 +-- !query analysis +Project [f1#x, sum(f1) OVER (PARTITION BY f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL] ++- Project [f1#x, f2#x, sum(f1) OVER (PARTITION BY f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL, sum(f1) OVER (PARTITION BY f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL] + +- Window [sum(f1#x) windowspecdefinition(f1#x, f2#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -1, 1)) AS sum(f1) OVER (PARTITION BY f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [f1#x], [f2#x ASC NULLS FIRST] + +- Project [f1#x, f2#x] + +- Filter (f1#x = f2#x) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[f1#x,f2#x] parquet + + +-- !query +select f1, sum(f1) over (partition by f1, f1 order by f2 +range between 2 preceding and 1 preceding) +from t1 where f1 = f2 +-- !query analysis +Project [f1#x, sum(f1) OVER (PARTITION BY f1, f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL] ++- Project [f1#x, f2#x, sum(f1) OVER (PARTITION BY f1, f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL, sum(f1) OVER (PARTITION BY f1, f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL] + +- Window [sum(f1#x) windowspecdefinition(f1#x, f1#x, f2#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -2, -1)) AS sum(f1) OVER (PARTITION BY f1, f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL], [f1#x, f1#x], [f2#x ASC NULLS FIRST] + +- Project [f1#x, f2#x] + +- Filter (f1#x = f2#x) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[f1#x,f2#x] parquet + + +-- !query +select f1, sum(f1) over (partition by f1, f2 order by f2 +range between 1 following and 2 following) +from t1 where f1 = f2 +-- !query analysis +Project [f1#x, sum(f1) OVER (PARTITION BY f1, f2 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 FOLLOWING AND 2 FOLLOWING)#xL] ++- Project [f1#x, f2#x, sum(f1) OVER (PARTITION BY f1, f2 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 FOLLOWING AND 2 FOLLOWING)#xL, sum(f1) OVER (PARTITION BY f1, f2 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 FOLLOWING AND 2 FOLLOWING)#xL] + +- Window [sum(f1#x) windowspecdefinition(f1#x, f2#x, f2#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, 1, 2)) AS sum(f1) OVER (PARTITION BY f1, f2 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 FOLLOWING AND 2 FOLLOWING)#xL], [f1#x, f2#x], [f2#x ASC NULLS FIRST] + +- Project [f1#x, f2#x] + +- Filter (f1#x = f2#x) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[f1#x,f2#x] parquet + + +-- !query +SELECT rank() OVER (ORDER BY length('abc')) +-- !query analysis +Project [RANK() OVER (ORDER BY length(abc) ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [RANK() OVER (ORDER BY length(abc) ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, RANK() OVER (ORDER BY length(abc) ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [rank(length(abc)) windowspecdefinition(length(abc) ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS RANK() OVER (ORDER BY length(abc) ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [length(abc) ASC NULLS FIRST] + +- Project + +- OneRowRelation + + +-- !query +SELECT * FROM empsalary WHERE row_number() OVER (ORDER BY salary) < 10 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1034", + "messageParameters" : { + "clauseName" : "WHERE" + } +} + + +-- !query +SELECT * FROM empsalary INNER JOIN tenk1 ON row_number() OVER (ORDER BY salary) < 10 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_EXPR_FOR_OPERATOR", + "messageParameters" : { + "invalidExprSqls" : "\"row_number() OVER (ORDER BY salary ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 84, + "fragment" : "INNER JOIN tenk1 ON row_number() OVER (ORDER BY salary) < 10" + } ] +} + + +-- !query +SELECT rank() OVER (ORDER BY 1), count(*) FROM empsalary GROUP BY 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_EXPR_FOR_OPERATOR", + "messageParameters" : { + "invalidExprSqls" : "\"RANK() OVER (ORDER BY 1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 58, + "stopIndex" : 67, + "fragment" : "GROUP BY 1" + } ] +} + + +-- !query +SELECT * FROM rank() OVER (ORDER BY random()) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'BY'", + "hint" : "" + } +} + + +-- !query +SELECT * FROM empsalary WHERE (rank() OVER (ORDER BY random())) > 10 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1034", + "messageParameters" : { + "clauseName" : "WHERE" + } +} + + +-- !query +SELECT * FROM empsalary WHERE rank() OVER (ORDER BY random()) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1034", + "messageParameters" : { + "clauseName" : "WHERE" + } +} + + +-- !query +select rank() OVER (PARTITION BY four, ORDER BY ten) FROM tenk1 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'BY'", + "hint" : ": extra input 'BY'" + } +} + + +-- !query +SELECT range(1, 100) OVER () FROM empsalary +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`range`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "range(1, 100) OVER ()" + } ] +} + + +-- !query +SELECT ntile(0) OVER (ORDER BY ten), ten, four FROM tenk1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + "sqlState" : "42K09", + "messageParameters" : { + "currentValue" : "0", + "exprName" : "buckets", + "sqlExpr" : "\"ntile(0)\"", + "valueRange" : "(0, 2147483647]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "ntile(0) OVER (ORDER BY ten)" + } ] +} + + +-- !query +SELECT nth_value(four, 0) OVER (ORDER BY ten), ten, four FROM tenk1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + "sqlState" : "42K09", + "messageParameters" : { + "currentValue" : "0L", + "exprName" : "offset", + "sqlExpr" : "\"nth_value(four, 0)\"", + "valueRange" : "(0, 9223372036854775807]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "nth_value(four, 0) OVER (ORDER BY ten)" + } ] +} + + +-- !query +DROP TABLE empsalary +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`empsalary`, false, false, false + + +-- !query +DROP TABLE datetimes +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`datetimes`, false, false, false + + +-- !query +DROP TABLE t1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t1`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part4.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part4.sql.out new file mode 100644 index 0000000000000..ed6cca796a5eb --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part4.sql.out @@ -0,0 +1,515 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query analysis +Project [i#x, avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [i#x, v#x, avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [avg(v#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [i#x, v#x] + +- SubqueryAlias t + +- Project [col1#x AS i#x, col2#x AS v#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query analysis +Project [i#x, avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [i#x, v#x, avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [avg(v#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [i#x, v#x] + +- SubqueryAlias t + +- Project [col1#x AS i#x, col2#x AS v#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query analysis +Project [i#x, avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [i#x, v#x, avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [avg(v#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [i#x, v#x] + +- SubqueryAlias t + +- Project [col1#x AS i#x, col2#x AS v#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.5),(2,2.5),(3,NULL),(4,NULL)) t(i,v) +-- !query analysis +Project [i#x, avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [i#x, v#x, avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [avg(v#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS avg(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [i#x, v#x] + +- SubqueryAlias t + +- Project [col1#x AS i#x, col2#x AS v#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query analysis +Project [i#x, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL] ++- Project [i#x, v#x, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL] + +- Window [sum(v#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL], [i#x ASC NULLS FIRST] + +- Project [i#x, v#x] + +- SubqueryAlias t + +- Project [col1#x AS i#x, col2#x AS v#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query analysis +Project [i#x, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL] ++- Project [i#x, v#x, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL] + +- Window [sum(v#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL], [i#x ASC NULLS FIRST] + +- Project [i#x, v#x] + +- SubqueryAlias t + +- Project [col1#x AS i#x, col2#x AS v#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query analysis +Project [i#x, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL] ++- Project [i#x, v#x, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL] + +- Window [sum(v#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL], [i#x ASC NULLS FIRST] + +- Project [i#x, v#x] + +- SubqueryAlias t + +- Project [col1#x AS i#x, col2#x AS v#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.1),(2,2.2),(3,NULL),(4,NULL)) t(i,v) +-- !query analysis +Project [i#x, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [i#x, v#x, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [sum(v#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [i#x, v#x] + +- SubqueryAlias t + +- Project [col1#x AS i#x, col2#x AS v#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT SUM(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.01),(2,2),(3,3)) v(i,n) +-- !query analysis +Project [sum(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [n#x, i#x, sum(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, sum(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [sum(n#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS sum(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [n#x, i#x] + +- SubqueryAlias v + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT i,COUNT(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query analysis +Project [i#x, count(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL] ++- Project [i#x, v#x, count(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL, count(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL] + +- Window [count(v#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS count(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL], [i#x ASC NULLS FIRST] + +- Project [i#x, v#x] + +- SubqueryAlias t + +- Project [col1#x AS i#x, col2#x AS v#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT i,COUNT(*) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query analysis +Project [i#x, count(1) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL] ++- Project [i#x, count(1) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL, count(1) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL] + +- Window [count(1) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS count(1) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#xL], [i#x ASC NULLS FIRST] + +- Project [i#x] + +- SubqueryAlias t + +- Project [col1#x AS i#x, col2#x AS v#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [var_pop(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [var_pop(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [var_pop(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [var_pop(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS var_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [var_samp(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [var_samp(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [var_samp(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [var_samp(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS var_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [variance(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [variance(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [variance(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [variance(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS variance(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query analysis +Project [stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [stddev_pop(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query analysis +Project [stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [stddev_pop(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query analysis +Project [stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [stddev_pop(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query analysis +Project [stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [stddev_pop(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS stddev_pop(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query analysis +Project [stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [stddev_samp(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query analysis +Project [stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [stddev_samp(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query analysis +Project [stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [stddev_samp(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query analysis +Project [stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [stddev_samp(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS stddev_samp(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [stddev(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [stddev(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [stddev(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query analysis +Project [stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] ++- Project [_w0#x, i#x, stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x, stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x] + +- Window [stddev(_w0#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), unboundedfollowing$())) AS stddev(n) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING)#x], [i#x ASC NULLS FIRST] + +- Project [cast(n#x as double) AS _w0#x, i#x] + +- SubqueryAlias r + +- Project [col1#x AS i#x, col2#x AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND CURRENT ROW) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query analysis +Project [i#x, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND CURRENT ROW)#xL] ++- Project [i#x, v#x, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND CURRENT ROW)#xL, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND CURRENT ROW)#xL] + +- Window [sum(v#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), currentrow$())) AS sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND CURRENT ROW)#xL], [i#x ASC NULLS FIRST] + +- Project [i#x, v#x] + +- SubqueryAlias t + +- Project [col1#x AS i#x, col2#x AS v#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query analysis +Project [i#x, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] ++- Project [i#x, v#x, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] + +- Window [sum(v#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), 1)) AS sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL], [i#x ASC NULLS FIRST] + +- Project [i#x, v#x] + +- SubqueryAlias t + +- Project [col1#x AS i#x, col2#x AS v#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,3),(4,4)) t(i,v) +-- !query analysis +Project [i#x, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL] ++- Project [i#x, v#x, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL, sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL] + +- Window [sum(v#x) windowspecdefinition(i#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, 1)) AS sum(v) OVER (ORDER BY i ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [i#x ASC NULLS FIRST] + +- Project [i#x, v#x] + +- SubqueryAlias t + +- Project [col1#x AS i#x, col2#x AS v#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT a, b, + SUM(b) OVER(ORDER BY A ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +FROM (VALUES(1,1),(2,2),(3,(cast('nan' as int))),(4,3),(5,4)) t(a,b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2331", + "messageParameters" : { + "msg" : "[CAST_INVALID_INPUT] The value 'nan' of the type \"STRING\" cannot be cast to \"INT\" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.\n== SQL(line 3, position 29) ==\nFROM (VALUES(1,1),(2,2),(3,(cast('nan' as int))),(4,3),(5,4)) t(a,b)\n ^^^^^^^^^^^^^^^^^^\n", + "sqlExpr" : "CAST('nan' AS INT)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 92, + "stopIndex" : 145, + "fragment" : "VALUES(1,1),(2,2),(3,(cast('nan' as int))),(4,3),(5,4)" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out new file mode 100644 index 0000000000000..bf37c45b0be5e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out @@ -0,0 +1,506 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +WITH q1(x,y) AS (SELECT 1,2) +SELECT * FROM q1, q1 AS q2 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias q1 +: +- Project [1#x AS x#x, 2#x AS y#x] +: +- Project [1 AS 1#x, 2 AS 2#x] +: +- OneRowRelation ++- Project [x#x, y#x, x#x, y#x] + +- Join Inner + :- SubqueryAlias q1 + : +- CTERelationRef xxxx, true, [x#x, y#x] + +- SubqueryAlias q2 + +- SubqueryAlias q1 + +- CTERelationRef xxxx, true, [x#x, y#x] + + +-- !query +SELECT count(*) FROM ( + WITH q1(x) AS (SELECT rand() FROM (SELECT EXPLODE(SEQUENCE(1, 5)))) + SELECT * FROM q1 + UNION + SELECT * FROM q1 +) ss +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +CREATE TABLE department ( + id INTEGER, -- department ID + parent_department INTEGER, -- upper department ID + name string -- department name +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`department`, false + + +-- !query +INSERT INTO department VALUES (0, NULL, 'ROOT') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO department VALUES (1, 0, 'A') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO department VALUES (2, 1, 'B') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO department VALUES (3, 2, 'C') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO department VALUES (4, 2, 'D') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO department VALUES (5, 0, 'E') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO department VALUES (6, 4, 'F') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO department VALUES (7, 5, 'G') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +CREATE TABLE tree( + id INTEGER, + parent_id INTEGER +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tree`, false + + +-- !query +INSERT INTO tree +VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), + (9,4), (10,4), (11,7), (12,7), (13,7), (14, 9), (15,11), (16,11) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tree, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/tree], Append, `spark_catalog`.`default`.`tree`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tree), [id, parent_id] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_id#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +create table graph( f int, t int, label string ) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`graph`, false + + +-- !query +insert into graph values + (1, 2, 'arc 1 -> 2'), + (1, 3, 'arc 1 -> 3'), + (2, 3, 'arc 2 -> 3'), + (1, 4, 'arc 1 -> 4'), + (4, 5, 'arc 4 -> 5'), + (5, 1, 'arc 5 -> 1') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/graph, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/graph], Append, `spark_catalog`.`default`.`graph`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/graph), [f, t, label] ++- Project [cast(col1#x as int) AS f#x, cast(col2#x as int) AS t#x, cast(col3#x as string) AS label#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +CREATE TABLE y (a INTEGER) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`y`, false + + +-- !query +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/y, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/y], Append, `spark_catalog`.`default`.`y`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/y), [a] ++- Project [cast(col#x as int) AS a#x] + +- Project [col#x] + +- Generate explode(sequence(1, 10, None, Some(America/Los_Angeles))), false, [col#x] + +- OneRowRelation + + +-- !query +DROP TABLE y +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`y`, false, false, false + + +-- !query +CREATE TABLE y (a INTEGER) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`y`, false + + +-- !query +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/y, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/y], Append, `spark_catalog`.`default`.`y`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/y), [a] ++- Project [cast(col#x as int) AS a#x] + +- Project [col#x] + +- Generate explode(sequence(1, 10, None, Some(America/Los_Angeles))), false, [col#x] + +- OneRowRelation + + +-- !query +with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte +: +- Project [42#x AS foo#x] +: +- Project [42 AS 42#x] +: +- OneRowRelation ++- Project [foo#x] + +- SubqueryAlias q + +- Project [foo#x] + +- SubqueryAlias cte + +- CTERelationRef xxxx, true, [foo#x] + + +-- !query +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM innermost + UNION SELECT 3) +) +SELECT * FROM outermost ORDER BY 1 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias innermost +: +- Project [2 AS 2#x] +: +- OneRowRelation +:- CTERelationDef xxxx, false +: +- SubqueryAlias outermost +: +- Project [1#x AS x#x] +: +- Distinct +: +- Union false, false +: :- Project [1 AS 1#x] +: : +- OneRowRelation +: +- Distinct +: +- Union false, false +: :- Project [2#x] +: : +- SubqueryAlias innermost +: : +- CTERelationRef xxxx, true, [2#x] +: +- Project [3 AS 3#x] +: +- OneRowRelation ++- Sort [x#x ASC NULLS FIRST], true + +- Project [x#x] + +- SubqueryAlias outermost + +- CTERelationRef xxxx, true, [x#x] + + +-- !query +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM outermost -- fail + UNION SELECT * FROM innermost) +) +SELECT * FROM outermost ORDER BY 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`outermost`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 96, + "stopIndex" : 104, + "fragment" : "outermost" + } ] +} + + +-- !query +CREATE TABLE withz USING parquet AS SELECT i AS k, CAST(i AS string) || ' v' AS v FROM (SELECT EXPLODE(SEQUENCE(1, 16, 3)) i) +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`withz`, ErrorIfExists, [k, v] + +- Project [i#x AS k#x, concat(cast(i#x as string), v) AS v#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [i#x] + +- Generate explode(sequence(1, 16, Some(3), Some(America/Los_Angeles))), false, [i#x] + +- OneRowRelation + + +-- !query +SELECT * FROM withz ORDER BY k +-- !query analysis +Sort [k#x ASC NULLS FIRST], true ++- Project [k#x, v#x] + +- SubqueryAlias spark_catalog.default.withz + +- Relation spark_catalog.default.withz[k#x,v#x] parquet + + +-- !query +DROP TABLE withz +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`withz`, false, false, false + + +-- !query +TRUNCATE TABLE y +-- !query analysis +TruncateTableCommand `spark_catalog`.`default`.`y` + + +-- !query +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 3)) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/y, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/y], Append, `spark_catalog`.`default`.`y`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/y), [a] ++- Project [cast(col#x as int) AS a#x] + +- Project [col#x] + +- Generate explode(sequence(1, 3, None, Some(America/Los_Angeles))), false, [col#x] + +- OneRowRelation + + +-- !query +CREATE TABLE yy (a INTEGER) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`yy`, false + + +-- !query +SELECT * FROM y +-- !query analysis +Project [a#x] ++- SubqueryAlias spark_catalog.default.y + +- Relation spark_catalog.default.y[a#x] parquet + + +-- !query +SELECT * FROM yy +-- !query analysis +Project [a#x] ++- SubqueryAlias spark_catalog.default.yy + +- Relation spark_catalog.default.yy[a#x] parquet + + +-- !query +SELECT * FROM y +-- !query analysis +Project [a#x] ++- SubqueryAlias spark_catalog.default.y + +- Relation spark_catalog.default.y[a#x] parquet + + +-- !query +SELECT * FROM yy +-- !query analysis +Project [a#x] ++- SubqueryAlias spark_catalog.default.yy + +- Relation spark_catalog.default.yy[a#x] parquet + + +-- !query +CREATE TABLE parent ( id int, val string ) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`parent`, false + + +-- !query +INSERT INTO parent VALUES ( 1, 'p1' ) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/parent, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/parent], Append, `spark_catalog`.`default`.`parent`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/parent), [id, val] ++- Project [cast(col1#x as int) AS id#x, cast(col2#x as string) AS val#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM parent +-- !query analysis +Project [id#x, val#x] ++- SubqueryAlias spark_catalog.default.parent + +- Relation spark_catalog.default.parent[id#x,val#x] parquet + + +-- !query +SELECT * FROM parent +-- !query analysis +Project [id#x, val#x] ++- SubqueryAlias spark_catalog.default.parent + +- Relation spark_catalog.default.parent[id#x,val#x] parquet + + +-- !query +create table foo (with baz) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_DATATYPE", + "sqlState" : "0A000", + "messageParameters" : { + "typeName" : "\"BAZ\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 26, + "fragment" : "baz" + } ] +} + + +-- !query +create table foo (with ordinality) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_DATATYPE", + "sqlState" : "0A000", + "messageParameters" : { + "typeName" : "\"ORDINALITY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 33, + "fragment" : "ordinality" + } ] +} + + +-- !query +with ordinality as (select 1 as x) select * from ordinality +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias ordinality +: +- Project [1 AS x#x] +: +- OneRowRelation ++- Project [x#x] + +- SubqueryAlias ordinality + +- CTERelationRef xxxx, true, [x#x] + + +-- !query +WITH test AS (SELECT 42) INSERT INTO test VALUES (1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`test`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 38, + "stopIndex" : 41, + "fragment" : "test" + } ] +} + + +-- !query +create table test (i int) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`test`, false + + +-- !query +with test as (select 42) insert into test select * from test +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test], Append, `spark_catalog`.`default`.`test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test), [i] ++- Project [cast(42#x as int) AS i#x] + +- Project [42#x] + +- SubqueryAlias test + +- Project [42 AS 42#x] + +- OneRowRelation + + +-- !query +select * from test +-- !query analysis +Project [i#x] ++- SubqueryAlias spark_catalog.default.test + +- Relation spark_catalog.default.test[i#x] parquet + + +-- !query +drop table test +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`test`, false, false, false + + +-- !query +DROP TABLE department +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`department`, false, false, false + + +-- !query +DROP TABLE tree +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tree`, false, false, false + + +-- !query +DROP TABLE graph +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`graph`, false, false, false + + +-- !query +DROP TABLE y +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`y`, false, false, false + + +-- !query +DROP TABLE yy +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`yy`, false, false, false + + +-- !query +DROP TABLE parent +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`parent`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/pred-pushdown.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/pred-pushdown.sql.out new file mode 100644 index 0000000000000..2c66e3a10dad9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/pred-pushdown.sql.out @@ -0,0 +1,57 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW tbl_a AS VALUES (1, 1), (2, 1), (3, 6) AS T(c1, c2) +-- !query analysis +CreateViewCommand `tbl_a`, VALUES (1, 1), (2, 1), (3, 6) AS T(c1, c2), false, true, LocalTempView, true + +- SubqueryAlias T + +- LocalRelation [c1#x, c2#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW tbl_b AS VALUES 1 AS T(c1) +-- !query analysis +CreateViewCommand `tbl_b`, VALUES 1 AS T(c1), false, true, LocalTempView, true + +- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SELECT * +FROM tbl_a + LEFT ANTI JOIN tbl_b ON ((tbl_a.c1 = tbl_a.c2) IS NULL OR tbl_a.c1 = tbl_a.c2) +-- !query analysis +Project [c1#x, c2#x] ++- Join LeftAnti, (isnull((c1#x = c2#x)) OR (c1#x = c2#x)) + :- SubqueryAlias tbl_a + : +- View (`tbl_a`, [c1#x,c2#x]) + : +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] + : +- SubqueryAlias T + : +- LocalRelation [c1#x, c2#x] + +- SubqueryAlias tbl_b + +- View (`tbl_b`, [c1#x]) + +- Project [cast(c1#x as int) AS c1#x] + +- SubqueryAlias T + +- LocalRelation [c1#x] + + +-- !query +SELECT l.c1, l.c2 +FROM tbl_a l +WHERE EXISTS (SELECT 1 FROM tbl_b r WHERE l.c1 = l.c2) OR l.c2 < 2 +-- !query analysis +Project [c1#x, c2#x] ++- Filter (exists#x [c1#x && c2#x] OR (c2#x < 2)) + : +- Project [1 AS 1#x] + : +- Filter (outer(c1#x) = outer(c2#x)) + : +- SubqueryAlias r + : +- SubqueryAlias tbl_b + : +- View (`tbl_b`, [c1#x]) + : +- Project [cast(c1#x as int) AS c1#x] + : +- SubqueryAlias T + : +- LocalRelation [c1#x] + +- SubqueryAlias l + +- SubqueryAlias tbl_a + +- View (`tbl_a`, [c1#x,c2#x]) + +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] + +- SubqueryAlias T + +- LocalRelation [c1#x, c2#x] 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 new file mode 100644 index 0000000000000..604d680a40544 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/predicate-functions.sql.out @@ -0,0 +1,370 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select 1 = 1 +-- !query analysis +Project [(1 = 1) AS (1 = 1)#x] ++- OneRowRelation + + +-- !query +select 1 = '1' +-- !query analysis +Project [(1 = cast(1 as int)) AS (1 = 1)#x] ++- OneRowRelation + + +-- !query +select 1.0 = '1' +-- !query analysis +Project [(cast(1.0 as double) = cast(1 as double)) AS (1.0 = 1)#x] ++- OneRowRelation + + +-- !query +select 1.5 = '1.51' +-- !query analysis +Project [(cast(1.5 as double) = cast(1.51 as double)) AS (1.5 = 1.51)#x] ++- OneRowRelation + + +-- !query +select 1 > '1' +-- !query analysis +Project [(1 > cast(1 as int)) AS (1 > 1)#x] ++- OneRowRelation + + +-- !query +select 2 > '1.0' +-- !query analysis +Project [(2 > cast(1.0 as int)) 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] ++- OneRowRelation + + +-- !query +select 2 > '2.2' +-- !query analysis +Project [(2 > cast(2.2 as int)) AS (2 > 2.2)#x] ++- OneRowRelation + + +-- !query +select '1.5' > 0.5 +-- !query analysis +Project [(cast(1.5 as double) > cast(0.5 as double)) AS (1.5 > 0.5)#x] ++- OneRowRelation + + +-- !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)) > to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles))) 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)) > 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] ++- OneRowRelation + + +-- !query +select 2 >= '1.0' +-- !query analysis +Project [(2 >= cast(1.0 as int)) 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] ++- OneRowRelation + + +-- !query +select 2.0 >= '2.2' +-- !query analysis +Project [(cast(2.0 as double) >= cast(2.2 as double)) AS (2.0 >= 2.2)#x] ++- OneRowRelation + + +-- !query +select '1.5' >= 0.5 +-- !query analysis +Project [(cast(1.5 as double) >= cast(0.5 as double)) AS (1.5 >= 0.5)#x] ++- OneRowRelation + + +-- !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)) >= to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles))) 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)) >= 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] ++- OneRowRelation + + +-- !query +select 2 < '1.0' +-- !query analysis +Project [(2 < cast(1.0 as int)) 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] ++- OneRowRelation + + +-- !query +select 2.0 < '2.2' +-- !query analysis +Project [(cast(2.0 as double) < cast(2.2 as double)) AS (2.0 < 2.2)#x] ++- OneRowRelation + + +-- !query +select 0.5 < '1.5' +-- !query analysis +Project [(cast(0.5 as double) < cast(1.5 as double)) AS (0.5 < 1.5)#x] ++- OneRowRelation + + +-- !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)) < to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles))) 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)) < 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] ++- OneRowRelation + + +-- !query +select 2 <= '1.0' +-- !query analysis +Project [(2 <= cast(1.0 as int)) 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] ++- OneRowRelation + + +-- !query +select 2.0 <= '2.2' +-- !query analysis +Project [(cast(2.0 as double) <= cast(2.2 as double)) AS (2.0 <= 2.2)#x] ++- OneRowRelation + + +-- !query +select 0.5 <= '1.5' +-- !query analysis +Project [(cast(0.5 as double) <= cast(1.5 as double)) AS (0.5 <= 1.5)#x] ++- OneRowRelation + + +-- !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)) <= to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles))) 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)) <= 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)) 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] ++- 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)) 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)) 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)) 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] ++- 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)) 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] ++- OneRowRelation + + +-- !query +select 1 in (1, 2, 3) +-- !query analysis +Project [1 IN (1,2,3) AS (1 IN (1, 2, 3))#x] ++- OneRowRelation + + +-- !query +select 1 in (1, 2, 3, null) +-- !query analysis +Project [cast(1 as int) IN (cast(1 as int),cast(2 as int),cast(3 as int),cast(null as int)) AS (1 IN (1, 2, 3, NULL))#x] ++- OneRowRelation + + +-- !query +select 1 in (1.0, 2.0, 3.0) +-- !query analysis +Project [cast(1 as decimal(11,1)) IN (cast(1.0 as decimal(11,1)),cast(2.0 as decimal(11,1)),cast(3.0 as decimal(11,1))) AS (1 IN (1.0, 2.0, 3.0))#x] ++- OneRowRelation + + +-- !query +select 1 in (1.0, 2.0, 3.0, null) +-- !query analysis +Project [cast(1 as decimal(11,1)) IN (cast(1.0 as decimal(11,1)),cast(2.0 as decimal(11,1)),cast(3.0 as decimal(11,1)),cast(null as decimal(11,1))) AS (1 IN (1.0, 2.0, 3.0, NULL))#x] ++- OneRowRelation + + +-- !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] ++- 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] ++- OneRowRelation + + +-- !query +select null in (1, 2, 3) +-- !query analysis +Project [cast(null as int) IN (cast(1 as int),cast(2 as int),cast(3 as int)) AS (NULL IN (1, 2, 3))#x] ++- OneRowRelation + + +-- !query +select null in (1, 2, null) +-- !query analysis +Project [cast(null as int) IN (cast(1 as int),cast(2 as int),cast(null as int)) AS (NULL IN (1, 2, NULL))#x] ++- OneRowRelation + + +-- !query +select 1 not in (1, 2, 3) +-- !query analysis +Project [NOT 1 IN (1,2,3) AS (NOT (1 IN (1, 2, 3)))#x] ++- OneRowRelation + + +-- !query +select 1 not in (1, 2, 3, null) +-- !query analysis +Project [NOT cast(1 as int) IN (cast(1 as int),cast(2 as int),cast(3 as int),cast(null as int)) AS (NOT (1 IN (1, 2, 3, NULL)))#x] ++- OneRowRelation + + +-- !query +select 1 not in (1.0, 2.0, 3.0) +-- !query analysis +Project [NOT cast(1 as decimal(11,1)) IN (cast(1.0 as decimal(11,1)),cast(2.0 as decimal(11,1)),cast(3.0 as decimal(11,1))) AS (NOT (1 IN (1.0, 2.0, 3.0)))#x] ++- OneRowRelation + + +-- !query +select 1 not in (1.0, 2.0, 3.0, null) +-- !query analysis +Project [NOT cast(1 as decimal(11,1)) IN (cast(1.0 as decimal(11,1)),cast(2.0 as decimal(11,1)),cast(3.0 as decimal(11,1)),cast(null as decimal(11,1))) AS (NOT (1 IN (1.0, 2.0, 3.0, NULL)))#x] ++- OneRowRelation + + +-- !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] ++- 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] ++- OneRowRelation + + +-- !query +select null not in (1, 2, 3) +-- !query analysis +Project [NOT cast(null as int) IN (cast(1 as int),cast(2 as int),cast(3 as int)) AS (NOT (NULL IN (1, 2, 3)))#x] ++- OneRowRelation + + +-- !query +select null not in (1, 2, null) +-- !query analysis +Project [NOT cast(null as int) IN (cast(1 as int),cast(2 as int),cast(null as int)) AS (NOT (NULL IN (1, 2, NULL)))#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/query_regex_column.sql.out new file mode 100644 index 0000000000000..f4953b27bfda2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/query_regex_column.sql.out @@ -0,0 +1,495 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.parser.quotedRegexColumnNames=false +-- !query analysis +SetCommand (spark.sql.parser.quotedRegexColumnNames,Some(false)) + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66") +AS testData(key, value1, value2) +-- !query analysis +CreateViewCommand `testData`, SELECT * FROM VALUES +(1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66") +AS testData(key, value1, value2), false, true, LocalTempView, true + +- Project [key#x, value1#x, value2#x] + +- SubqueryAlias testData + +- LocalRelation [key#x, value1#x, value2#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES +(1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4) +AS testData2(A, B, c, d) +-- !query analysis +CreateViewCommand `testData2`, SELECT * FROM VALUES +(1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4) +AS testData2(A, B, c, d), false, true, LocalTempView, true + +- Project [A#x, B#x, c#x, d#x] + +- SubqueryAlias testData2 + +- LocalRelation [A#x, B#x, c#x, d#x] + + +-- !query +SELECT `(a)?+.+` FROM testData2 WHERE a = 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`(a)?+.+`", + "proposal" : "`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "`(a)?+.+`" + } ] +} + + +-- !query +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t`.`(a)?+.+`", + "proposal" : "`t`.`A`, `t`.`B`, `t`.`c`, `t`.`d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "t.`(a)?+.+`" + } ] +} + + +-- !query +SELECT `(a|b)` FROM testData2 WHERE a = 2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`(a|b)`", + "proposal" : "`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "`(a|b)`" + } ] +} + + +-- !query +SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`(a|b)?+.+`", + "proposal" : "`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "`(a|b)?+.+`" + } ] +} + + +-- !query +SELECT SUM(`(a|b)?+.+`) FROM testData2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`(a|b)?+.+`", + "proposal" : "`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 22, + "fragment" : "`(a|b)?+.+`" + } ] +} + + +-- !query +SELECT SUM(`(a)`) FROM testData2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`(a)`", + "proposal" : "`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 16, + "fragment" : "`(a)`" + } ] +} + + +-- !query +set spark.sql.parser.quotedRegexColumnNames=true +-- !query analysis +SetCommand (spark.sql.parser.quotedRegexColumnNames,Some(true)) + + +-- !query +SELECT `(a)?+.+` FROM testData2 WHERE a = 1 +-- !query analysis +Project [B#x, c#x, d#x] ++- Filter (a#x = 1) + +- SubqueryAlias testdata2 + +- View (`testData2`, [A#x,B#x,c#x,d#x]) + +- Project [cast(A#x as int) AS A#x, cast(B#x as int) AS B#x, cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + +- Project [A#x, B#x, c#x, d#x] + +- SubqueryAlias testData2 + +- LocalRelation [A#x, B#x, c#x, d#x] + + +-- !query +SELECT `(A)?+.+` FROM testData2 WHERE a = 1 +-- !query analysis +Project [B#x, c#x, d#x] ++- Filter (a#x = 1) + +- SubqueryAlias testdata2 + +- View (`testData2`, [A#x,B#x,c#x,d#x]) + +- Project [cast(A#x as int) AS A#x, cast(B#x as int) AS B#x, cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + +- Project [A#x, B#x, c#x, d#x] + +- SubqueryAlias testData2 + +- LocalRelation [A#x, B#x, c#x, d#x] + + +-- !query +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 +-- !query analysis +Project [B#x, c#x, d#x] ++- Filter (a#x = 1) + +- SubqueryAlias t + +- SubqueryAlias testdata2 + +- View (`testData2`, [A#x,B#x,c#x,d#x]) + +- Project [cast(A#x as int) AS A#x, cast(B#x as int) AS B#x, cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + +- Project [A#x, B#x, c#x, d#x] + +- SubqueryAlias testData2 + +- LocalRelation [A#x, B#x, c#x, d#x] + + +-- !query +SELECT t.`(A)?+.+` FROM testData2 t WHERE a = 1 +-- !query analysis +Project [B#x, c#x, d#x] ++- Filter (a#x = 1) + +- SubqueryAlias t + +- SubqueryAlias testdata2 + +- View (`testData2`, [A#x,B#x,c#x,d#x]) + +- Project [cast(A#x as int) AS A#x, cast(B#x as int) AS B#x, cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + +- Project [A#x, B#x, c#x, d#x] + +- SubqueryAlias testData2 + +- LocalRelation [A#x, B#x, c#x, d#x] + + +-- !query +SELECT `(a|B)` FROM testData2 WHERE a = 2 +-- !query analysis +Project [A#x, B#x] ++- Filter (a#x = 2) + +- SubqueryAlias testdata2 + +- View (`testData2`, [A#x,B#x,c#x,d#x]) + +- Project [cast(A#x as int) AS A#x, cast(B#x as int) AS B#x, cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + +- Project [A#x, B#x, c#x, d#x] + +- SubqueryAlias testData2 + +- LocalRelation [A#x, B#x, c#x, d#x] + + +-- !query +SELECT `(A|b)` FROM testData2 WHERE a = 2 +-- !query analysis +Project [A#x, B#x] ++- Filter (a#x = 2) + +- SubqueryAlias testdata2 + +- View (`testData2`, [A#x,B#x,c#x,d#x]) + +- Project [cast(A#x as int) AS A#x, cast(B#x as int) AS B#x, cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + +- Project [A#x, B#x, c#x, d#x] + +- SubqueryAlias testData2 + +- LocalRelation [A#x, B#x, c#x, d#x] + + +-- !query +SELECT `(a|B)?+.+` FROM testData2 WHERE a = 2 +-- !query analysis +Project [c#x, d#x] ++- Filter (a#x = 2) + +- SubqueryAlias testdata2 + +- View (`testData2`, [A#x,B#x,c#x,d#x]) + +- Project [cast(A#x as int) AS A#x, cast(B#x as int) AS B#x, cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + +- Project [A#x, B#x, c#x, d#x] + +- SubqueryAlias testData2 + +- LocalRelation [A#x, B#x, c#x, d#x] + + +-- !query +SELECT `(A|b)?+.+` FROM testData2 WHERE a = 2 +-- !query analysis +Project [c#x, d#x] ++- Filter (a#x = 2) + +- SubqueryAlias testdata2 + +- View (`testData2`, [A#x,B#x,c#x,d#x]) + +- Project [cast(A#x as int) AS A#x, cast(B#x as int) AS B#x, cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + +- Project [A#x, B#x, c#x, d#x] + +- SubqueryAlias testData2 + +- LocalRelation [A#x, B#x, c#x, d#x] + + +-- !query +SELECT `(e|f)` FROM testData2 +-- !query analysis +Project ++- SubqueryAlias testdata2 + +- View (`testData2`, [A#x,B#x,c#x,d#x]) + +- Project [cast(A#x as int) AS A#x, cast(B#x as int) AS B#x, cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + +- Project [A#x, B#x, c#x, d#x] + +- SubqueryAlias testData2 + +- LocalRelation [A#x, B#x, c#x, d#x] + + +-- !query +SELECT t.`(e|f)` FROM testData2 t +-- !query analysis +Project ++- SubqueryAlias t + +- SubqueryAlias testdata2 + +- View (`testData2`, [A#x,B#x,c#x,d#x]) + +- Project [cast(A#x as int) AS A#x, cast(B#x as int) AS B#x, cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + +- Project [A#x, B#x, c#x, d#x] + +- SubqueryAlias testData2 + +- LocalRelation [A#x, B#x, c#x, d#x] + + +-- !query +SELECT p.`(KEY)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 +-- !query analysis +Project [value1#x, value2#x, b#x, A#x, c#x, d#x] ++- Filter (key#x < 3) + +- Join Inner, (key#x = a#x) + :- SubqueryAlias p + : +- SubqueryAlias testdata + : +- View (`testData`, [key#x,value1#x,value2#x]) + : +- Project [cast(key#x as int) AS key#x, cast(value1#x as string) AS value1#x, cast(value2#x as string) AS value2#x] + : +- Project [key#x, value1#x, value2#x] + : +- SubqueryAlias testData + : +- LocalRelation [key#x, value1#x, value2#x] + +- SubqueryAlias testdata2 + +- View (`testData2`, [A#x,B#x,c#x,d#x]) + +- Project [cast(A#x as int) AS A#x, cast(B#x as int) AS B#x, cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + +- Project [A#x, B#x, c#x, d#x] + +- SubqueryAlias testData2 + +- LocalRelation [A#x, B#x, c#x, d#x] + + +-- !query +SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 +-- !query analysis +Project [value1#x, value2#x, b#x, A#x, c#x, d#x] ++- Filter (key#x < 3) + +- Join Inner, (key#x = a#x) + :- SubqueryAlias p + : +- SubqueryAlias testdata + : +- View (`testData`, [key#x,value1#x,value2#x]) + : +- Project [cast(key#x as int) AS key#x, cast(value1#x as string) AS value1#x, cast(value2#x as string) AS value2#x] + : +- Project [key#x, value1#x, value2#x] + : +- SubqueryAlias testData + : +- LocalRelation [key#x, value1#x, value2#x] + +- SubqueryAlias testdata2 + +- View (`testData2`, [A#x,B#x,c#x,d#x]) + +- Project [cast(A#x as int) AS A#x, cast(B#x as int) AS B#x, cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + +- Project [A#x, B#x, c#x, d#x] + +- SubqueryAlias testData2 + +- LocalRelation [A#x, B#x, c#x, d#x] + + +-- !query +set spark.sql.caseSensitive=true +-- !query analysis +SetCommand (spark.sql.caseSensitive,Some(true)) + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testdata3 AS SELECT * FROM VALUES +(0, 1), (1, 2), (2, 3), (3, 4) +AS testdata3(a, b) +-- !query analysis +CreateViewCommand `testdata3`, SELECT * FROM VALUES +(0, 1), (1, 2), (2, 3), (3, 4) +AS testdata3(a, b), false, true, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias testdata3 + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT `(A)?+.+` FROM testdata3 +-- !query analysis +Project [a#x, b#x] ++- SubqueryAlias testdata3 + +- View (`testdata3`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testdata3 + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT `(a)?+.+` FROM testdata3 +-- !query analysis +Project [b#x] ++- SubqueryAlias testdata3 + +- View (`testdata3`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testdata3 + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT `(A)?+.+` FROM testdata3 WHERE a > 1 +-- !query analysis +Project [a#x, b#x] ++- Filter (a#x > 1) + +- SubqueryAlias testdata3 + +- View (`testdata3`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testdata3 + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT `(a)?+.+` FROM testdata3 where `a` > 1 +-- !query analysis +Project [b#x] ++- Filter (a#x > 1) + +- SubqueryAlias testdata3 + +- View (`testdata3`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testdata3 + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT SUM(`a`) FROM testdata3 +-- !query analysis +Aggregate [sum(a#x) AS sum(a)#xL] ++- SubqueryAlias testdata3 + +- View (`testdata3`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testdata3 + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT SUM(`(a)`) FROM testdata3 +-- !query analysis +Aggregate [sum(a#x) AS sum(a)#xL] ++- SubqueryAlias testdata3 + +- View (`testdata3`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testdata3 + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT SUM(`(a)?+.+`) FROM testdata3 +-- !query analysis +Aggregate [sum(b#x) AS sum(b)#xL] ++- SubqueryAlias testdata3 + +- View (`testdata3`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testdata3 + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT SUM(a) FROM testdata3 GROUP BY `a` +-- !query analysis +Aggregate [a#x], [sum(a#x) AS sum(a)#xL] ++- SubqueryAlias testdata3 + +- View (`testdata3`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testdata3 + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT SUM(a) FROM testdata3 GROUP BY `(a)` +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`(a)`", + "proposal" : "`testdata3`.`a`, `testdata3`.`b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 43, + "fragment" : "`(a)`" + } ] +} + + +-- !query +SELECT SUM(a) FROM testdata3 GROUP BY `(a)?+.+` +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`(a)?+.+`", + "proposal" : "`testdata3`.`a`, `testdata3`.`b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 47, + "fragment" : "`(a)?+.+`" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out new file mode 100644 index 0000000000000..de5dc246c0e87 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out @@ -0,0 +1,95 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT rand(0) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT rand(cast(3 / 7 AS int)) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT rand(NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT rand(cast(NULL AS int)) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT rand(1.0) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "1", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"rand(1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "rand(1.0)" + } ] +} + + +-- !query +SELECT randn(0L) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT randn(cast(3 / 7 AS long)) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT randn(NULL) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT randn(cast(NULL AS long)) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT rand('1') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"rand(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "rand('1')" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/regexp-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/regexp-functions.sql.out new file mode 100644 index 0000000000000..1d30315e3f919 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/regexp-functions.sql.out @@ -0,0 +1,516 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+') +-- !query analysis +Project [regexp_extract(1a 2b 14m, \d+, 1) AS regexp_extract(1a 2b 14m, \d+, 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', 0) +-- !query analysis +Project [regexp_extract(1a 2b 14m, \d+, 0) AS regexp_extract(1a 2b 14m, \d+, 0)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', 1) +-- !query analysis +Project [regexp_extract(1a 2b 14m, \d+, 1) AS regexp_extract(1a 2b 14m, \d+, 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', 2) +-- !query analysis +Project [regexp_extract(1a 2b 14m, \d+, 2) AS regexp_extract(1a 2b 14m, \d+, 2)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', -1) +-- !query analysis +Project [regexp_extract(1a 2b 14m, \d+, -1) AS regexp_extract(1a 2b 14m, \d+, -1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)?', 1) +-- !query analysis +Project [regexp_extract(1a 2b 14m, (\d+)?, 1) AS regexp_extract(1a 2b 14m, (\d+)?, 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract('a b m', '(\\d+)?', 1) +-- !query analysis +Project [regexp_extract(a b m, (\d+)?, 1) AS regexp_extract(a b m, (\d+)?, 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)') +-- !query analysis +Project [regexp_extract(1a 2b 14m, (\d+)([a-z]+), 1) AS regexp_extract(1a 2b 14m, (\d+)([a-z]+), 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 0) +-- !query analysis +Project [regexp_extract(1a 2b 14m, (\d+)([a-z]+), 0) AS regexp_extract(1a 2b 14m, (\d+)([a-z]+), 0)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 1) +-- !query analysis +Project [regexp_extract(1a 2b 14m, (\d+)([a-z]+), 1) AS regexp_extract(1a 2b 14m, (\d+)([a-z]+), 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 2) +-- !query analysis +Project [regexp_extract(1a 2b 14m, (\d+)([a-z]+), 2) AS regexp_extract(1a 2b 14m, (\d+)([a-z]+), 2)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 3) +-- !query analysis +Project [regexp_extract(1a 2b 14m, (\d+)([a-z]+), 3) AS regexp_extract(1a 2b 14m, (\d+)([a-z]+), 3)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', -1) +-- !query analysis +Project [regexp_extract(1a 2b 14m, (\d+)([a-z]+), -1) AS regexp_extract(1a 2b 14m, (\d+)([a-z]+), -1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query analysis +Project [regexp_extract(1a 2b 14m, (\d+)?([a-z]+), 1) AS regexp_extract(1a 2b 14m, (\d+)?([a-z]+), 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract('a b m', '(\\d+)?([a-z]+)', 1) +-- !query analysis +Project [regexp_extract(a b m, (\d+)?([a-z]+), 1) AS regexp_extract(a b m, (\d+)?([a-z]+), 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(?l)') +-- !query analysis +Project [regexp_extract(1a 2b 14m, (?l), 1) AS regexp_extract(1a 2b 14m, (?l), 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+') +-- !query analysis +Project [regexp_extract_all(1a 2b 14m, \d+, 1) AS regexp_extract_all(1a 2b 14m, \d+, 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 0) +-- !query analysis +Project [regexp_extract_all(1a 2b 14m, \d+, 0) AS regexp_extract_all(1a 2b 14m, \d+, 0)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 1) +-- !query analysis +Project [regexp_extract_all(1a 2b 14m, \d+, 1) AS regexp_extract_all(1a 2b 14m, \d+, 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 2) +-- !query analysis +Project [regexp_extract_all(1a 2b 14m, \d+, 2) AS regexp_extract_all(1a 2b 14m, \d+, 2)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', -1) +-- !query analysis +Project [regexp_extract_all(1a 2b 14m, \d+, -1) AS regexp_extract_all(1a 2b 14m, \d+, -1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?', 1) +-- !query analysis +Project [regexp_extract_all(1a 2b 14m, (\d+)?, 1) AS regexp_extract_all(1a 2b 14m, (\d+)?, 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?', 1) +-- !query analysis +Project [regexp_extract_all(a 2b 14m, (\d+)?, 1) AS regexp_extract_all(a 2b 14m, (\d+)?, 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)') +-- !query analysis +Project [regexp_extract_all(1a 2b 14m, (\d+)([a-z]+), 1) AS regexp_extract_all(1a 2b 14m, (\d+)([a-z]+), 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 0) +-- !query analysis +Project [regexp_extract_all(1a 2b 14m, (\d+)([a-z]+), 0) AS regexp_extract_all(1a 2b 14m, (\d+)([a-z]+), 0)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 1) +-- !query analysis +Project [regexp_extract_all(1a 2b 14m, (\d+)([a-z]+), 1) AS regexp_extract_all(1a 2b 14m, (\d+)([a-z]+), 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 2) +-- !query analysis +Project [regexp_extract_all(1a 2b 14m, (\d+)([a-z]+), 2) AS regexp_extract_all(1a 2b 14m, (\d+)([a-z]+), 2)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 3) +-- !query analysis +Project [regexp_extract_all(1a 2b 14m, (\d+)([a-z]+), 3) AS regexp_extract_all(1a 2b 14m, (\d+)([a-z]+), 3)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', -1) +-- !query analysis +Project [regexp_extract_all(1a 2b 14m, (\d+)([a-z]+), -1) AS regexp_extract_all(1a 2b 14m, (\d+)([a-z]+), -1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query analysis +Project [regexp_extract_all(1a 2b 14m, (\d+)?([a-z]+), 1) AS regexp_extract_all(1a 2b 14m, (\d+)?([a-z]+), 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query analysis +Project [regexp_extract_all(a 2b 14m, (\d+)?([a-z]+), 1) AS regexp_extract_all(a 2b 14m, (\d+)?([a-z]+), 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_extract_all('abc', col0, 1) FROM VALUES('], [') AS t(col0) +-- !query analysis +Project [regexp_extract_all(abc, col0#x, 1) AS regexp_extract_all(abc, col0, 1)#x] ++- SubqueryAlias t + +- LocalRelation [col0#x] + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something') +-- !query analysis +Project [regexp_replace(healthy, wealthy, and wise, \w+thy, something, 1) AS regexp_replace(healthy, wealthy, and wise, \w+thy, something, 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', -2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + "sqlState" : "42K09", + "messageParameters" : { + "currentValue" : "-2", + "exprName" : "position", + "sqlExpr" : "\"regexp_replace(healthy, wealthy, and wise, \\w+thy, something, -2)\"", + "valueRange" : "(0, 2147483647]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "regexp_replace('healthy, wealthy, and wise', '\\\\w+thy', 'something', -2)" + } ] +} + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 0) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + "sqlState" : "42K09", + "messageParameters" : { + "currentValue" : "0", + "exprName" : "position", + "sqlExpr" : "\"regexp_replace(healthy, wealthy, and wise, \\w+thy, something, 0)\"", + "valueRange" : "(0, 2147483647]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "regexp_replace('healthy, wealthy, and wise', '\\\\w+thy', 'something', 0)" + } ] +} + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 1) +-- !query analysis +Project [regexp_replace(healthy, wealthy, and wise, \w+thy, something, 1) AS regexp_replace(healthy, wealthy, and wise, \w+thy, something, 1)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 2) +-- !query analysis +Project [regexp_replace(healthy, wealthy, and wise, \w+thy, something, 2) AS regexp_replace(healthy, wealthy, and wise, \w+thy, something, 2)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 8) +-- !query analysis +Project [regexp_replace(healthy, wealthy, and wise, \w+thy, something, 8) AS regexp_replace(healthy, wealthy, and wise, \w+thy, something, 8)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', 26) +-- !query analysis +Project [regexp_replace(healthy, wealthy, and wise, \w, something, 26) AS regexp_replace(healthy, wealthy, and wise, \w, something, 26)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', 27) +-- !query analysis +Project [regexp_replace(healthy, wealthy, and wise, \w, something, 27) AS regexp_replace(healthy, wealthy, and wise, \w, something, 27)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', 30) +-- !query analysis +Project [regexp_replace(healthy, wealthy, and wise, \w, something, 30) AS regexp_replace(healthy, wealthy, and wise, \w, something, 30)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', null) +-- !query analysis +Project [regexp_replace(healthy, wealthy, and wise, \w, something, cast(null as int)) AS regexp_replace(healthy, wealthy, and wise, \w, something, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_like('1a 2b 14m', '\\d+b') +-- !query analysis +Project [RLIKE(1a 2b 14m, \d+b) AS REGEXP_LIKE(1a 2b 14m, \d+b)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_like('1a 2b 14m', '[a-z]+b') +-- !query analysis +Project [RLIKE(1a 2b 14m, [a-z]+b) AS REGEXP_LIKE(1a 2b 14m, [a-z]+b)#x] ++- OneRowRelation + + +-- !query +SELECT regexp('1a 2b 14m', '\\d+b') +-- !query analysis +Project [RLIKE(1a 2b 14m, \d+b) AS REGEXP(1a 2b 14m, \d+b)#x] ++- OneRowRelation + + +-- !query +SELECT regexp('1a 2b 14m', '[a-z]+b') +-- !query analysis +Project [RLIKE(1a 2b 14m, [a-z]+b) AS REGEXP(1a 2b 14m, [a-z]+b)#x] ++- OneRowRelation + + +-- !query +SELECT rlike('1a 2b 14m', '\\d+b') +-- !query analysis +Project [RLIKE(1a 2b 14m, \d+b) AS RLIKE(1a 2b 14m, \d+b)#x] ++- OneRowRelation + + +-- !query +SELECT rlike('1a 2b 14m', '[a-z]+b') +-- !query analysis +Project [RLIKE(1a 2b 14m, [a-z]+b) AS RLIKE(1a 2b 14m, [a-z]+b)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_count('1a 2b 14m', '\\d+') +-- !query analysis +Project [regexp_count(1a 2b 14m, \d+) AS regexp_count(1a 2b 14m, \d+)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_count('1a 2b 14m', 'mmm') +-- !query analysis +Project [regexp_count(1a 2b 14m, mmm) AS regexp_count(1a 2b 14m, mmm)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_count('the fox', 'FOX') +-- !query analysis +Project [regexp_count(the fox, FOX) AS regexp_count(the fox, FOX)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_count('the fox', '(?i)FOX') +-- !query analysis +Project [regexp_count(the fox, (?i)FOX) AS regexp_count(the fox, (?i)FOX)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_count('passwd7 plain A1234 a1234', '(?=[^ ]*[a-z])(?=[^ ]*[0-9])[^ ]+') +-- !query analysis +Project [regexp_count(passwd7 plain A1234 a1234, (?=[^ ]*[a-z])(?=[^ ]*[0-9])[^ ]+) AS regexp_count(passwd7 plain A1234 a1234, (?=[^ ]*[a-z])(?=[^ ]*[0-9])[^ ]+)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_count(null, 'abc') +-- !query analysis +Project [regexp_count(cast(null as string), abc) AS regexp_count(NULL, abc)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_count('abc', null) +-- !query analysis +Project [regexp_count(abc, cast(null as string)) AS regexp_count(abc, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_substr('1a 2b 14m', '\\d+') +-- !query analysis +Project [regexp_substr(1a 2b 14m, \d+) AS regexp_substr(1a 2b 14m, \d+)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_substr('1a 2b 14m', '\\d+ ') +-- !query analysis +Project [regexp_substr(1a 2b 14m, \d+ ) AS regexp_substr(1a 2b 14m, \d+ )#x] ++- OneRowRelation + + +-- !query +SELECT regexp_substr('1a 2b 14m', '\\d+(a|b|m)') +-- !query analysis +Project [regexp_substr(1a 2b 14m, \d+(a|b|m)) AS regexp_substr(1a 2b 14m, \d+(a|b|m))#x] ++- OneRowRelation + + +-- !query +SELECT regexp_substr('1a 2b 14m', '\\d{2}(a|b|m)') +-- !query analysis +Project [regexp_substr(1a 2b 14m, \d{2}(a|b|m)) AS regexp_substr(1a 2b 14m, \d{2}(a|b|m))#x] ++- OneRowRelation + + +-- !query +SELECT regexp_substr('1a 2b 14m', '') +-- !query analysis +Project [regexp_substr(1a 2b 14m, ) AS regexp_substr(1a 2b 14m, )#x] ++- OneRowRelation + + +-- !query +SELECT regexp_substr('Spark', null) +-- !query analysis +Project [regexp_substr(Spark, cast(null as string)) AS regexp_substr(Spark, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_substr(null, '.*') +-- !query analysis +Project [regexp_substr(cast(null as string), .*) AS regexp_substr(NULL, .*)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_instr('abc', 'b') +-- !query analysis +Project [regexp_instr(abc, b, 0) AS regexp_instr(abc, b, 0)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_instr('abc', 'x') +-- !query analysis +Project [regexp_instr(abc, x, 0) AS regexp_instr(abc, x, 0)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_instr('ABC', '(?-i)b') +-- !query analysis +Project [regexp_instr(ABC, (?-i)b, 0) AS regexp_instr(ABC, (?-i)b, 0)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_instr('1a 2b 14m', '\\d{2}(a|b|m)') +-- !query analysis +Project [regexp_instr(1a 2b 14m, \d{2}(a|b|m), 0) AS regexp_instr(1a 2b 14m, \d{2}(a|b|m), 0)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_instr('abc', null) +-- !query analysis +Project [regexp_instr(abc, cast(null as string), 0) AS regexp_instr(abc, NULL, 0)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_instr(null, 'b') +-- !query analysis +Project [regexp_instr(cast(null as string), b, 0) AS regexp_instr(NULL, b, 0)#x] ++- OneRowRelation + + +-- !query +SELECT regexp_instr('abc', col0, 1) FROM VALUES(') ?') AS t(col0) +-- !query analysis +Project [regexp_instr(abc, col0#x, 1) AS regexp_instr(abc, col0, 1)#x] ++- SubqueryAlias t + +- LocalRelation [col0#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/show-create-table.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/show-create-table.sql.out new file mode 100644 index 0000000000000..bfc49ef55ccb6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/show-create-table.sql.out @@ -0,0 +1,295 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false + + +-- !query +SHOW CREATE TABLE tbl +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tbl, V1Table(default.tbl), [a#x, b#x, c#x] + + +-- !query +DROP TABLE tbl +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +OPTIONS ('a' 1, 'password' = 'password') +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false + + +-- !query +SHOW CREATE TABLE tbl +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tbl, V1Table(default.tbl), [a#x, b#x, c#x] + + +-- !query +DROP TABLE tbl +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +OPTIONS ('path' '/path/to/table') +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false + + +-- !query +SHOW CREATE TABLE tbl +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tbl, V1Table(default.tbl), [a#x, b#x, c#x] + + +-- !query +DROP TABLE tbl +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +LOCATION '/path/to/table' +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false + + +-- !query +SHOW CREATE TABLE tbl +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tbl, V1Table(default.tbl), [a#x, b#x, c#x] + + +-- !query +DROP TABLE tbl +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +PARTITIONED BY (a) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false + + +-- !query +SHOW CREATE TABLE tbl +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tbl, V1Table(default.tbl), [b#x, c#x, a#x] + + +-- !query +DROP TABLE tbl +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false + + +-- !query +SHOW CREATE TABLE tbl +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tbl, V1Table(default.tbl), [a#x, b#x, c#x] + + +-- !query +DROP TABLE tbl +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false + + +-- !query +CREATE TABLE tbl (a INT DEFAULT 42, b STRING DEFAULT 'abc, def', c INT DEFAULT 42) USING parquet +COMMENT 'This is a comment' +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false + + +-- !query +SHOW CREATE TABLE tbl +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tbl, V1Table(default.tbl), [a#x, b#x, c#x] + + +-- !query +DROP TABLE tbl +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +COMMENT 'This is a comment' +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false + + +-- !query +SHOW CREATE TABLE tbl +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tbl, V1Table(default.tbl), [a#x, b#x, c#x] + + +-- !query +DROP TABLE tbl +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +TBLPROPERTIES ('a' = '1', 'password' = 'password') +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false + + +-- !query +SHOW CREATE TABLE tbl +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tbl, V1Table(default.tbl), [a#x, b#x, c#x] + + +-- !query +DROP TABLE tbl +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false + + +-- !query +CREATE TABLE tbl (a REAL, b NUMERIC, c NUMERIC(10), d NUMERIC(10,1)) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false + + +-- !query +SHOW CREATE TABLE tbl +-- !query analysis +ShowCreateTable false, [createtab_stmt#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tbl, V1Table(default.tbl), [a#x, b#x, c#x, d#x] + + +-- !query +DROP TABLE tbl +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false + + +-- !query +CREATE VIEW view_SPARK_30302 (aaa, bbb) +AS SELECT a, b FROM tbl +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`view_SPARK_30302`, [(aaa,None), (bbb,None)], SELECT a, b FROM tbl, false, false, PersistedView, true + +- Project [a#x, b#x] + +- SubqueryAlias spark_catalog.default.tbl + +- Relation spark_catalog.default.tbl[a#x,b#x,c#x] parquet + + +-- !query +SHOW CREATE TABLE view_SPARK_30302 AS SERDE +-- !query analysis +ShowCreateTableAsSerdeCommand `spark_catalog`.`default`.`view_spark_30302`, [createtab_stmt#x] + + +-- !query +SHOW CREATE TABLE view_SPARK_30302 +-- !query analysis +ShowCreateTableCommand `spark_catalog`.`default`.`view_spark_30302`, [createtab_stmt#x] + + +-- !query +DROP VIEW view_SPARK_30302 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`view_SPARK_30302`, false, true, false + + +-- !query +CREATE VIEW view_SPARK_30302 (aaa COMMENT 'comment with \'quoted text\' for aaa', bbb) +COMMENT 'This is a comment with \'quoted text\' for view' +AS SELECT a, b FROM tbl +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`view_SPARK_30302`, [(aaa,Some(comment with 'quoted text' for aaa)), (bbb,None)], This is a comment with 'quoted text' for view, SELECT a, b FROM tbl, false, false, PersistedView, true + +- Project [a#x, b#x] + +- SubqueryAlias spark_catalog.default.tbl + +- Relation spark_catalog.default.tbl[a#x,b#x,c#x] parquet + + +-- !query +SHOW CREATE TABLE view_SPARK_30302 AS SERDE +-- !query analysis +ShowCreateTableAsSerdeCommand `spark_catalog`.`default`.`view_spark_30302`, [createtab_stmt#x] + + +-- !query +SHOW CREATE TABLE view_SPARK_30302 +-- !query analysis +ShowCreateTableCommand `spark_catalog`.`default`.`view_spark_30302`, [createtab_stmt#x] + + +-- !query +DROP VIEW view_SPARK_30302 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`view_SPARK_30302`, false, true, false + + +-- !query +CREATE VIEW view_SPARK_30302 (aaa, bbb) +TBLPROPERTIES ('a' = '1', 'b' = '2') +AS SELECT a, b FROM tbl +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`view_SPARK_30302`, [(aaa,None), (bbb,None)], [a=1, b=2], SELECT a, b FROM tbl, false, false, PersistedView, true + +- Project [a#x, b#x] + +- SubqueryAlias spark_catalog.default.tbl + +- Relation spark_catalog.default.tbl[a#x,b#x,c#x] parquet + + +-- !query +SHOW CREATE TABLE view_SPARK_30302 AS SERDE +-- !query analysis +ShowCreateTableAsSerdeCommand `spark_catalog`.`default`.`view_spark_30302`, [createtab_stmt#x] + + +-- !query +SHOW CREATE TABLE view_SPARK_30302 +-- !query analysis +ShowCreateTableCommand `spark_catalog`.`default`.`view_spark_30302`, [createtab_stmt#x] + + +-- !query +DROP VIEW view_SPARK_30302 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`view_SPARK_30302`, false, true, false + + +-- !query +DROP TABLE tbl +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out new file mode 100644 index 0000000000000..e59436be9ce50 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out @@ -0,0 +1,219 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE DATABASE showdb +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb] + + +-- !query +USE showdb +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb] + + +-- !query +CREATE TABLE show_t1(a String, b Int, c String, d String) USING parquet PARTITIONED BY (c, d) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`showdb`.`show_t1`, false + + +-- !query +ALTER TABLE show_t1 ADD PARTITION (c='Us', d=1) +-- !query analysis +AlterTableAddPartitionCommand `spark_catalog`.`showdb`.`show_t1`, [(Map(c -> Us, d -> 1),None)], false + + +-- !query +CREATE TABLE show_t2(b String, d Int) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`showdb`.`show_t2`, false + + +-- !query +CREATE TEMPORARY VIEW show_t3(e int) USING parquet +-- !query analysis +CreateTempViewUsing [tableIdent:`show_t3` StructType(StructField(e,IntegerType,true)) replace:false provider:parquet Map() + + +-- !query +CREATE GLOBAL TEMP VIEW show_t4 AS SELECT 1 as col1 +-- !query analysis +CreateViewCommand `show_t4`, SELECT 1 as col1, false, false, GlobalTempView, true + +- Project [1 AS col1#x] + +- OneRowRelation + + +-- !query +SHOW TABLES +-- !query analysis +ShowTables [namespace#x, tableName#x, isTemporary#x] ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb] + + +-- !query +SHOW TABLES IN showdb +-- !query analysis +ShowTables [namespace#x, tableName#x, isTemporary#x] ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb] + + +-- !query +SHOW TABLES 'show_t*' +-- !query analysis +ShowTables show_t*, [namespace#x, tableName#x, isTemporary#x] ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb] + + +-- !query +SHOW TABLES LIKE 'show_t1*|show_t2*' +-- !query analysis +ShowTables show_t1*|show_t2*, [namespace#x, tableName#x, isTemporary#x] ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb] + + +-- !query +SHOW TABLES IN showdb 'show_t*' +-- !query analysis +ShowTables show_t*, [namespace#x, tableName#x, isTemporary#x] ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb] + + +-- !query +SHOW TABLES IN showdb LIKE 'show_t*' +-- !query analysis +ShowTables show_t*, [namespace#x, tableName#x, isTemporary#x] ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb] + + +-- !query +SHOW TABLE EXTENDED LIKE 'show_t*' +-- !query analysis +ShowTablesCommand showdb, show_t*, [namespace#x, tableName#x, isTemporary#x, information#x], true + + +-- !query +SHOW TABLE EXTENDED +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "end of input", + "hint" : "" + } +} + + +-- !query +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us', d=1) +-- !query analysis +ShowTablesCommand showdb, show_t1, [namespace#x, tableName#x, isTemporary#x, information#x], true, Map(c -> Us, d -> 1) + + +-- !query +SHOW TABLE EXTENDED PARTITION(c='Us', d=1) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'PARTITION'", + "hint" : "" + } +} + + +-- !query +SHOW TABLE EXTENDED LIKE 'show_t*' PARTITION(c='Us', d=1) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`showdb`.`show_t*`" + } +} + + +-- !query +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1232", + "messageParameters" : { + "partitionColumnNames" : "c, d", + "specKeys" : "c", + "tableName" : "`spark_catalog`.`showdb`.`show_t1`" + } +} + + +-- !query +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(a='Us', d=1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1231", + "messageParameters" : { + "key" : "a", + "tblName" : "`spark_catalog`.`showdb`.`show_t1`" + } +} + + +-- !query +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Ch', d=1) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException +{ + "errorClass" : "PARTITIONS_NOT_FOUND", + "sqlState" : "428FT", + "messageParameters" : { + "partitionList" : "PARTITION (`c` = Ch, `d` = 1)", + "tableName" : "`showdb`.`show_t1`" + } +} + + +-- !query +DROP TABLE show_t1 +-- !query analysis +DropTableCommand `spark_catalog`.`showdb`.`show_t1`, false, false, false + + +-- !query +DROP TABLE show_t2 +-- !query analysis +DropTableCommand `spark_catalog`.`showdb`.`show_t2`, false, false, false + + +-- !query +DROP VIEW show_t3 +-- !query analysis +DropTempViewCommand show_t3 + + +-- !query +DROP VIEW global_temp.show_t4 +-- !query analysis +DropTempViewCommand global_temp.show_t4 + + +-- !query +USE default +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [default] + + +-- !query +DROP DATABASE showdb +-- !query analysis +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/show-tblproperties.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/show-tblproperties.sql.out new file mode 100644 index 0000000000000..a9ecb72ba6fbc --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/show-tblproperties.sql.out @@ -0,0 +1,85 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +TBLPROPERTIES('p1'='v1', 'p2'='v2', password = 'password') +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tbl`, false + + +-- !query +SHOW TBLPROPERTIES tbl +-- !query analysis +ShowTableProperties [key#x, value#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tbl, V1Table(default.tbl), [a#x, b#x, c#x] + + +-- !query +SHOW TBLPROPERTIES tbl("p1") +-- !query analysis +ShowTableProperties p1, [key#x, value#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tbl, V1Table(default.tbl), [a#x, b#x, c#x] + + +-- !query +SHOW TBLPROPERTIES tbl("p3") +-- !query analysis +ShowTableProperties p3, [key#x, value#x] ++- ResolvedTable V2SessionCatalog(spark_catalog), default.tbl, V1Table(default.tbl), [a#x, b#x, c#x] + + +-- !query +DROP TABLE tbl +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false + + +-- !query +CREATE VIEW view TBLPROPERTIES('p1'='v1', 'p2'='v2') AS SELECT 1 AS c1 +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`view`, [p1=v1, p2=v2], SELECT 1 AS c1, false, false, PersistedView, true + +- Project [1 AS c1#x] + +- OneRowRelation + + +-- !query +SHOW TBLPROPERTIES view +-- !query analysis +ShowTablePropertiesCommand `spark_catalog`.`default`.`view`, [key#x, value#x] + + +-- !query +SHOW TBLPROPERTIES view("p1") +-- !query analysis +ShowTablePropertiesCommand `spark_catalog`.`default`.`view`, p1, [key#x, value#x] + + +-- !query +SHOW TBLPROPERTIES view("p3") +-- !query analysis +ShowTablePropertiesCommand `spark_catalog`.`default`.`view`, p3, [key#x, value#x] + + +-- !query +DROP VIEW view +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`view`, false, true, false + + +-- !query +CREATE TEMPORARY VIEW tv AS SELECT 1 AS c1 +-- !query analysis +CreateViewCommand `tv`, SELECT 1 AS c1, false, false, LocalTempView, true + +- Project [1 AS c1#x] + +- OneRowRelation + + +-- !query +SHOW TBLPROPERTIES tv +-- !query analysis +ShowTablePropertiesCommand `tv`, [key#x, value#x] + + +-- !query +DROP VIEW tv +-- !query analysis +DropTempViewCommand tv diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/show-views.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/show-views.sql.out new file mode 100644 index 0000000000000..c8f2c6f9cc029 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/show-views.sql.out @@ -0,0 +1,139 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE DATABASE showdb +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb] + + +-- !query +USE showdb +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb] + + +-- !query +CREATE TABLE tbl(a STRING, b INT, c STRING, d STRING) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`showdb`.`tbl`, false + + +-- !query +CREATE VIEW view_1 AS SELECT * FROM tbl +-- !query analysis +CreateViewCommand `spark_catalog`.`showdb`.`view_1`, SELECT * FROM tbl, false, false, PersistedView, true + +- Project [a#x, b#x, c#x, d#x] + +- SubqueryAlias spark_catalog.showdb.tbl + +- Relation spark_catalog.showdb.tbl[a#x,b#x,c#x,d#x] parquet + + +-- !query +CREATE VIEW view_2 AS SELECT * FROM tbl WHERE c='a' +-- !query analysis +CreateViewCommand `spark_catalog`.`showdb`.`view_2`, SELECT * FROM tbl WHERE c='a', false, false, PersistedView, true + +- Project [a#x, b#x, c#x, d#x] + +- Filter (c#x = a) + +- SubqueryAlias spark_catalog.showdb.tbl + +- Relation spark_catalog.showdb.tbl[a#x,b#x,c#x,d#x] parquet + + +-- !query +CREATE GLOBAL TEMP VIEW view_3 AS SELECT 1 as col1 +-- !query analysis +CreateViewCommand `view_3`, SELECT 1 as col1, false, false, GlobalTempView, true + +- Project [1 AS col1#x] + +- OneRowRelation + + +-- !query +CREATE TEMPORARY VIEW view_4(e INT) USING parquet +-- !query analysis +CreateTempViewUsing [tableIdent:`view_4` StructType(StructField(e,IntegerType,true)) replace:false provider:parquet Map() + + +-- !query +SHOW VIEWS +-- !query analysis +ShowViewsCommand showdb, [namespace#x, viewName#x, isTemporary#x] + + +-- !query +SHOW VIEWS FROM showdb +-- !query analysis +ShowViewsCommand showdb, [namespace#x, viewName#x, isTemporary#x] + + +-- !query +SHOW VIEWS IN showdb +-- !query analysis +ShowViewsCommand showdb, [namespace#x, viewName#x, isTemporary#x] + + +-- !query +SHOW VIEWS IN global_temp +-- !query analysis +ShowViewsCommand global_temp, [namespace#x, viewName#x, isTemporary#x] + + +-- !query +SHOW VIEWS 'view_*' +-- !query analysis +ShowViewsCommand showdb, view_*, [namespace#x, viewName#x, isTemporary#x] + + +-- !query +SHOW VIEWS LIKE 'view_1*|view_2*' +-- !query analysis +ShowViewsCommand showdb, view_1*|view_2*, [namespace#x, viewName#x, isTemporary#x] + + +-- !query +SHOW VIEWS IN showdb 'view_*' +-- !query analysis +ShowViewsCommand showdb, view_*, [namespace#x, viewName#x, isTemporary#x] + + +-- !query +SHOW VIEWS IN showdb LIKE 'view_*' +-- !query analysis +ShowViewsCommand showdb, view_*, [namespace#x, viewName#x, isTemporary#x] + + +-- !query +SHOW VIEWS IN wrongdb LIKE 'view_*' +-- !query analysis +org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +{ + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42704", + "messageParameters" : { + "schemaName" : "`wrongdb`" + } +} + + +-- !query +DROP VIEW global_temp.view_3 +-- !query analysis +DropTempViewCommand global_temp.view_3 + + +-- !query +DROP VIEW view_4 +-- !query analysis +DropTempViewCommand view_4 + + +-- !query +USE default +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [default] + + +-- !query +DROP DATABASE showdb CASCADE +-- !query analysis +DropNamespace false, true ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/show_columns.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/show_columns.sql.out new file mode 100644 index 0000000000000..b174a5870f6da --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/show_columns.sql.out @@ -0,0 +1,218 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE DATABASE showdb +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb] + + +-- !query +USE showdb +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb] + + +-- !query +CREATE TABLE showcolumn1 (col1 int, `col 2` int) USING json +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`showdb`.`showcolumn1`, false + + +-- !query +CREATE TABLE showcolumn2 (price int, qty int, year int, month int) USING parquet partitioned by (year, month) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`showdb`.`showcolumn2`, false + + +-- !query +CREATE TEMPORARY VIEW showColumn3 (col3 int, `col 4` int) USING json +-- !query analysis +CreateTempViewUsing [tableIdent:`showColumn3` StructType(StructField(col3,IntegerType,true),StructField(col 4,IntegerType,true)) replace:false provider:json Map() + + +-- !query +CREATE GLOBAL TEMP VIEW showColumn4 AS SELECT 1 as col1, 'abc' as `col 5` +-- !query analysis +CreateViewCommand `showColumn4`, SELECT 1 as col1, 'abc' as `col 5`, false, false, GlobalTempView, true + +- Project [1 AS col1#x, abc AS col 5#x] + +- OneRowRelation + + +-- !query +SHOW COLUMNS IN showcolumn1 +-- !query analysis +ShowColumnsCommand `spark_catalog`.`showdb`.`showcolumn1`, [col_name#x] + + +-- !query +SHOW COLUMNS IN showdb.showcolumn1 +-- !query analysis +ShowColumnsCommand `spark_catalog`.`showdb`.`showcolumn1`, [col_name#x] + + +-- !query +SHOW COLUMNS IN showcolumn1 FROM showdb +-- !query analysis +ShowColumnsCommand showdb, `spark_catalog`.`showdb`.`showcolumn1`, [col_name#x] + + +-- !query +SHOW COLUMNS IN showcolumn2 IN showdb +-- !query analysis +ShowColumnsCommand showdb, `spark_catalog`.`showdb`.`showcolumn2`, [col_name#x] + + +-- !query +SHOW COLUMNS IN badtable FROM showdb +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`showdb`.`badtable`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 24, + "fragment" : "badtable" + } ] +} + + +-- !query +SHOW COLUMNS IN showdb.showcolumn1 from SHOWDB +-- !query analysis +ShowColumnsCommand SHOWDB, `spark_catalog`.`showdb`.`showcolumn1`, [col_name#x] + + +-- !query +SHOW COLUMNS IN showdb.showcolumn1 FROM baddb +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1057", + "messageParameters" : { + "dbA" : "baddb", + "dbB" : "showdb" + } +} + + +-- !query +SHOW COLUMNS IN showcolumn3 +-- !query analysis +ShowColumnsCommand `showcolumn3`, [col_name#x] + + +-- !query +SHOW COLUMNS IN showdb.showcolumn3 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`showdb`.`showcolumn3`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 34, + "fragment" : "showdb.showcolumn3" + } ] +} + + +-- !query +SHOW COLUMNS IN showcolumn3 FROM showdb +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`showdb`.`showcolumn3`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 27, + "fragment" : "showcolumn3" + } ] +} + + +-- !query +SHOW COLUMNS IN showcolumn4 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`showcolumn4`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 27, + "fragment" : "showcolumn4" + } ] +} + + +-- !query +SHOW COLUMNS IN global_temp.showcolumn4 +-- !query analysis +ShowColumnsCommand `global_temp`.`showcolumn4`, [col_name#x] + + +-- !query +SHOW COLUMNS IN showcolumn4 FROM global_temp +-- !query analysis +ShowColumnsCommand global_temp, `global_temp`.`showcolumn4`, [col_name#x] + + +-- !query +DROP TABLE showcolumn1 +-- !query analysis +DropTableCommand `spark_catalog`.`showdb`.`showcolumn1`, false, false, false + + +-- !query +DROP TABLE showColumn2 +-- !query analysis +DropTableCommand `spark_catalog`.`showdb`.`showColumn2`, false, false, false + + +-- !query +DROP VIEW showcolumn3 +-- !query analysis +DropTempViewCommand showcolumn3 + + +-- !query +DROP VIEW global_temp.showcolumn4 +-- !query analysis +DropTempViewCommand global_temp.showcolumn4 + + +-- !query +use default +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [default] + + +-- !query +DROP DATABASE showdb +-- !query analysis +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [showdb] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-compatibility-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-compatibility-functions.sql.out new file mode 100644 index 0000000000000..d3e150200f513 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-compatibility-functions.sql.out @@ -0,0 +1,118 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT ifnull(null, 'x'), ifnull('y', 'x'), ifnull(null, null) +-- !query analysis +Project [ifnull(null, x) AS ifnull(NULL, x)#x, ifnull(y, x) AS ifnull(y, x)#x, ifnull(null, null) AS ifnull(NULL, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT nullif('x', 'x'), nullif('x', 'y') +-- !query analysis +Project [nullif(x, x) AS nullif(x, x)#x, nullif(x, y) AS nullif(x, y)#x] ++- OneRowRelation + + +-- !query +SELECT nvl(null, 'x'), nvl('y', 'x'), nvl(null, null) +-- !query analysis +Project [nvl(null, x) AS nvl(NULL, x)#x, nvl(y, x) AS nvl(y, x)#x, nvl(null, null) AS nvl(NULL, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT nvl2(null, 'x', 'y'), nvl2('n', 'x', 'y'), nvl2(null, null, null) +-- !query analysis +Project [nvl2(null, x, y) AS nvl2(NULL, x, y)#x, nvl2(n, x, y) AS nvl2(n, x, y)#x, nvl2(null, null, null) AS nvl2(NULL, NULL, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT ifnull(1, 2.1d), ifnull(null, 2.1d) +-- !query analysis +Project [ifnull(1, 2.1) AS ifnull(1, 2.1)#x, ifnull(null, 2.1) AS ifnull(NULL, 2.1)#x] ++- OneRowRelation + + +-- !query +SELECT nullif(1, 2.1d), nullif(1, 1.0d) +-- !query analysis +Project [nullif(1, 2.1) AS nullif(1, 2.1)#x, nullif(1, 1.0) AS nullif(1, 1.0)#x] ++- OneRowRelation + + +-- !query +SELECT nvl(1, 2.1d), nvl(null, 2.1d) +-- !query analysis +Project [nvl(1, 2.1) AS nvl(1, 2.1)#x, nvl(null, 2.1) AS nvl(NULL, 2.1)#x] ++- OneRowRelation + + +-- !query +SELECT nvl2(null, 1, 2.1d), nvl2('n', 1, 2.1d) +-- !query analysis +Project [nvl2(null, 1, 2.1) AS nvl2(NULL, 1, 2.1)#x, nvl2(n, 1, 2.1) AS nvl2(n, 1, 2.1)#x] ++- OneRowRelation + + +-- !query +SELECT boolean(1), tinyint(1), smallint(1), int(1), bigint(1) +-- !query analysis +Project [cast(1 as boolean) AS 1#x, cast(1 as tinyint) AS 1#x, cast(1 as smallint) AS 1#x, cast(1 as int) AS 1#x, cast(1 as bigint) AS 1#xL] ++- OneRowRelation + + +-- !query +SELECT float(1), double(1), decimal(1) +-- !query analysis +Project [cast(1 as float) AS 1#x, cast(1 as double) AS 1#x, cast(1 as decimal(10,0)) AS 1#x] ++- OneRowRelation + + +-- !query +SELECT date("2014-04-04"), timestamp(date("2014-04-04")) +-- !query analysis +Project [cast(2014-04-04 as date) AS 2014-04-04#x, cast(cast(2014-04-04 as date) as timestamp) AS 2014-04-04#x] ++- OneRowRelation + + +-- !query +SELECT string(1, 2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`string`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "string(1, 2)" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW tempView1 AS VALUES (1, NAMED_STRUCT('col1', 'gamma', 'col2', 'delta')) AS T(id, st) +-- !query analysis +CreateViewCommand `tempView1`, VALUES (1, NAMED_STRUCT('col1', 'gamma', 'col2', 'delta')) AS T(id, st), false, false, LocalTempView, true + +- SubqueryAlias T + +- LocalRelation [id#x, st#x] + + +-- !query +SELECT nvl(st.col1, "value"), count(*) FROM from tempView1 GROUP BY nvl(st.col1, "value") +-- !query analysis +Aggregate [nvl(st#x.col1, value)], [nvl(st#x.col1, value) AS nvl(st.col1, value)#x, count(1) AS FROM#xL] ++- SubqueryAlias tempview1 + +- View (`tempView1`, [id#x,st#x]) + +- Project [cast(id#x as int) AS id#x, cast(st#x as struct) AS st#x] + +- SubqueryAlias T + +- LocalRelation [id#x, st#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/string-functions.sql.out new file mode 100644 index 0000000000000..9a66950d36508 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/string-functions.sql.out @@ -0,0 +1,1474 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select concat_ws() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 0", + "functionName" : "`concat_ws`" + } +} + + +-- !query +select format_string() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 0", + "functionName" : "`format_string`" + } +} + + +-- !query +select 'a' || 'b' || 'c' +-- !query analysis +Project [concat(concat(a, b), c) AS concat(concat(a, b), c)#x] ++- OneRowRelation + + +-- !query +select replace('abc', 'b', '123') +-- !query analysis +Project [replace(abc, b, 123) AS replace(abc, b, 123)#x] ++- OneRowRelation + + +-- !query +select replace('abc', 'b') +-- !query analysis +Project [replace(abc, b, ) AS replace(abc, b, )#x] ++- OneRowRelation + + +-- !query +select length(uuid()), (uuid() <> uuid()) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aaads', null) +-- !query analysis +Project [locate(bar, foobarbar, 1) AS locate(bar, foobarbar, 1)#x, position(cast(null as string), foobarbar, 1) AS position(NULL, foobarbar, 1)#x, position(aaads, cast(null as string), 1) AS position(aaads, NULL, 1)#x] ++- OneRowRelation + + +-- !query +select left("abcd", 2), left("abcd", 5), left("abcd", '2'), left("abcd", null) +-- !query analysis +Project [left(abcd, 2) AS left(abcd, 2)#x, left(abcd, 5) AS left(abcd, 5)#x, left(abcd, cast(2 as int)) AS left(abcd, 2)#x, left(abcd, cast(null as int)) AS left(abcd, NULL)#x] ++- OneRowRelation + + +-- !query +select left(null, -2) +-- !query analysis +Project [left(cast(null as string), -2) AS left(NULL, -2)#x] ++- OneRowRelation + + +-- !query +select left("abcd", -2), left("abcd", 0), left("abcd", 'a') +-- !query analysis +Project [left(abcd, -2) AS left(abcd, -2)#x, left(abcd, 0) AS left(abcd, 0)#x, left(abcd, cast(a as int)) AS left(abcd, a)#x] ++- OneRowRelation + + +-- !query +select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null) +-- !query analysis +Project [right(abcd, 2) AS right(abcd, 2)#x, right(abcd, 5) AS right(abcd, 5)#x, right(abcd, cast(2 as int)) AS right(abcd, 2)#x, right(abcd, cast(null as int)) AS right(abcd, NULL)#x] ++- OneRowRelation + + +-- !query +select right(null, -2) +-- !query analysis +Project [right(cast(null as string), -2) AS right(NULL, -2)#x] ++- OneRowRelation + + +-- !query +select right("abcd", -2), right("abcd", 0), right("abcd", 'a') +-- !query analysis +Project [right(abcd, -2) AS right(abcd, -2)#x, right(abcd, 0) AS right(abcd, 0)#x, right(abcd, cast(a as int)) AS right(abcd, a)#x] ++- OneRowRelation + + +-- !query +SELECT split('aa1cc2ee3', '[1-9]+') +-- !query analysis +Project [split(aa1cc2ee3, [1-9]+, -1) AS split(aa1cc2ee3, [1-9]+, -1)#x] ++- OneRowRelation + + +-- !query +SELECT split('aa1cc2ee3', '[1-9]+', 2) +-- !query analysis +Project [split(aa1cc2ee3, [1-9]+, 2) AS split(aa1cc2ee3, [1-9]+, 2)#x] ++- OneRowRelation + + +-- !query +SELECT split('hello', '') +-- !query analysis +Project [split(hello, , -1) AS split(hello, , -1)#x] ++- OneRowRelation + + +-- !query +SELECT split('', '') +-- !query analysis +Project [split(, , -1) AS split(, , -1)#x] ++- OneRowRelation + + +-- !query +SELECT split('abc', null) +-- !query analysis +Project [split(abc, cast(null as string), -1) AS split(abc, NULL, -1)#x] ++- OneRowRelation + + +-- !query +SELECT split(null, 'b') +-- !query analysis +Project [split(cast(null as string), b, -1) AS split(NULL, b, -1)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '.', 2) +-- !query analysis +Project [split_part(11.12.13, ., 2) AS split_part(11.12.13, ., 2)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '.', -1) +-- !query analysis +Project [split_part(11.12.13, ., -1) AS split_part(11.12.13, ., -1)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '.', -3) +-- !query analysis +Project [split_part(11.12.13, ., -3) AS split_part(11.12.13, ., -3)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '', 1) +-- !query analysis +Project [split_part(11.12.13, , 1) AS split_part(11.12.13, , 1)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11ab12ab13', 'ab', 1) +-- !query analysis +Project [split_part(11ab12ab13, ab, 1) AS split_part(11ab12ab13, ab, 1)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '.', 0) +-- !query analysis +Project [split_part(11.12.13, ., 0) AS split_part(11.12.13, ., 0)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '.', 4) +-- !query analysis +Project [split_part(11.12.13, ., 4) AS split_part(11.12.13, ., 4)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '.', 5) +-- !query analysis +Project [split_part(11.12.13, ., 5) AS split_part(11.12.13, ., 5)#x] ++- OneRowRelation + + +-- !query +SELECT split_part('11.12.13', '.', -5) +-- !query analysis +Project [split_part(11.12.13, ., -5) AS split_part(11.12.13, ., -5)#x] ++- OneRowRelation + + +-- !query +SELECT split_part(null, '.', 1) +-- !query analysis +Project [split_part(cast(null as string), ., 1) AS split_part(NULL, ., 1)#x] ++- OneRowRelation + + +-- !query +SELECT split_part(str, delimiter, partNum) FROM VALUES ('11.12.13', '.', 3) AS v1(str, delimiter, partNum) +-- !query analysis +Project [split_part(str#x, delimiter#x, partNum#x) AS split_part(str, delimiter, partNum)#x] ++- SubqueryAlias v1 + +- LocalRelation [str#x, delimiter#x, partNum#x] + + +-- !query +SELECT substr('Spark SQL', 5) +-- !query analysis +Project [substr(Spark SQL, 5, 2147483647) AS substr(Spark SQL, 5, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substr('Spark SQL', -3) +-- !query analysis +Project [substr(Spark SQL, -3, 2147483647) AS substr(Spark SQL, -3, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substr('Spark SQL', 5, 1) +-- !query analysis +Project [substr(Spark SQL, 5, 1) AS substr(Spark SQL, 5, 1)#x] ++- OneRowRelation + + +-- !query +SELECT substr('Spark SQL' from 5) +-- !query analysis +Project [substring(Spark SQL, 5, 2147483647) AS substring(Spark SQL, 5, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substr('Spark SQL' from -3) +-- !query analysis +Project [substring(Spark SQL, -3, 2147483647) AS substring(Spark SQL, -3, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substr('Spark SQL' from 5 for 1) +-- !query analysis +Project [substring(Spark SQL, 5, 1) AS substring(Spark SQL, 5, 1)#x] ++- OneRowRelation + + +-- !query +SELECT substring('Spark SQL', 5) +-- !query analysis +Project [substring(Spark SQL, 5, 2147483647) AS substring(Spark SQL, 5, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substring('Spark SQL', -3) +-- !query analysis +Project [substring(Spark SQL, -3, 2147483647) AS substring(Spark SQL, -3, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substring('Spark SQL', 5, 1) +-- !query analysis +Project [substring(Spark SQL, 5, 1) AS substring(Spark SQL, 5, 1)#x] ++- OneRowRelation + + +-- !query +SELECT substring('Spark SQL' from 5) +-- !query analysis +Project [substring(Spark SQL, 5, 2147483647) AS substring(Spark SQL, 5, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substring('Spark SQL' from -3) +-- !query analysis +Project [substring(Spark SQL, -3, 2147483647) AS substring(Spark SQL, -3, 2147483647)#x] ++- OneRowRelation + + +-- !query +SELECT substring('Spark SQL' from 5 for 1) +-- !query analysis +Project [substring(Spark SQL, 5, 1) AS substring(Spark SQL, 5, 1)#x] ++- OneRowRelation + + +-- !query +SELECT trim(" xyz "), ltrim(" xyz "), rtrim(" xyz ") +-- !query analysis +Project [trim( xyz , None) AS trim( xyz )#x, ltrim( xyz , None) AS ltrim( xyz )#x, rtrim( xyz , None) AS rtrim( xyz )#x] ++- OneRowRelation + + +-- !query +SELECT trim(BOTH 'xyz' FROM 'yxTomxx'), trim('xyz' FROM 'yxTomxx') +-- !query analysis +Project [trim(yxTomxx, Some(xyz)) AS TRIM(BOTH xyz FROM yxTomxx)#x, trim(yxTomxx, Some(xyz)) AS TRIM(BOTH xyz FROM yxTomxx)#x] ++- OneRowRelation + + +-- !query +SELECT trim(BOTH 'x' FROM 'xxxbarxxx'), trim('x' FROM 'xxxbarxxx') +-- !query analysis +Project [trim(xxxbarxxx, Some(x)) AS TRIM(BOTH x FROM xxxbarxxx)#x, trim(xxxbarxxx, Some(x)) AS TRIM(BOTH x FROM xxxbarxxx)#x] ++- OneRowRelation + + +-- !query +SELECT trim(LEADING 'xyz' FROM 'zzzytest') +-- !query analysis +Project [ltrim(zzzytest, Some(xyz)) AS TRIM(LEADING xyz FROM zzzytest)#x] ++- OneRowRelation + + +-- !query +SELECT trim(LEADING 'xyz' FROM 'zzzytestxyz') +-- !query analysis +Project [ltrim(zzzytestxyz, Some(xyz)) AS TRIM(LEADING xyz FROM zzzytestxyz)#x] ++- OneRowRelation + + +-- !query +SELECT trim(LEADING 'xy' FROM 'xyxXxyLAST WORD') +-- !query analysis +Project [ltrim(xyxXxyLAST WORD, Some(xy)) AS TRIM(LEADING xy FROM xyxXxyLAST WORD)#x] ++- OneRowRelation + + +-- !query +SELECT trim(TRAILING 'xyz' FROM 'testxxzx') +-- !query analysis +Project [rtrim(testxxzx, Some(xyz)) AS TRIM(TRAILING xyz FROM testxxzx)#x] ++- OneRowRelation + + +-- !query +SELECT trim(TRAILING 'xyz' FROM 'xyztestxxzx') +-- !query analysis +Project [rtrim(xyztestxxzx, Some(xyz)) AS TRIM(TRAILING xyz FROM xyztestxxzx)#x] ++- OneRowRelation + + +-- !query +SELECT trim(TRAILING 'xy' FROM 'TURNERyxXxy') +-- !query analysis +Project [rtrim(TURNERyxXxy, Some(xy)) AS TRIM(TRAILING xy FROM TURNERyxXxy)#x] ++- OneRowRelation + + +-- !query +SELECT btrim('xyxtrimyyx', 'xy') +-- !query analysis +Project [btrim(xyxtrimyyx, xy) AS btrim(xyxtrimyyx, xy)#x] ++- OneRowRelation + + +-- !query +SELECT btrim(encode(" xyz ", 'utf-8')) +-- !query analysis +Project [btrim(encode( xyz , utf-8)) AS btrim(encode( xyz , utf-8))#x] ++- OneRowRelation + + +-- !query +SELECT btrim(encode('yxTomxx', 'utf-8'), encode('xyz', 'utf-8')) +-- !query analysis +Project [btrim(encode(yxTomxx, utf-8), encode(xyz, utf-8)) AS btrim(encode(yxTomxx, utf-8), encode(xyz, utf-8))#x] ++- OneRowRelation + + +-- !query +SELECT btrim(encode('xxxbarxxx', 'utf-8'), encode('x', 'utf-8')) +-- !query analysis +Project [btrim(encode(xxxbarxxx, utf-8), encode(x, utf-8)) AS btrim(encode(xxxbarxxx, utf-8), encode(x, utf-8))#x] ++- OneRowRelation + + +-- !query +SELECT lpad('hi', 'invalid_length') +-- !query analysis +Project [lpad(hi, cast(invalid_length as int), ) AS lpad(hi, invalid_length, )#x] ++- OneRowRelation + + +-- !query +SELECT rpad('hi', 'invalid_length') +-- !query analysis +Project [rpad(hi, cast(invalid_length as int), ) AS rpad(hi, invalid_length, )#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex(''), 5)) +-- !query analysis +Project [hex(lpad(lpad, unhex(, false), 5, 0x00)) AS hex(lpad(unhex(), 5, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aabb'), 5)) +-- !query analysis +Project [hex(lpad(lpad, unhex(aabb, false), 5, 0x00)) AS hex(lpad(unhex(aabb), 5, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aabbcc'), 2)) +-- !query analysis +Project [hex(lpad(lpad, unhex(aabbcc, false), 2, 0x00)) AS hex(lpad(unhex(aabbcc), 2, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('123'), 2)) +-- !query analysis +Project [hex(lpad(lpad, unhex(123, false), 2, 0x00)) AS hex(lpad(unhex(123), 2, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('12345'), 2)) +-- !query analysis +Project [hex(lpad(lpad, unhex(12345, false), 2, 0x00)) AS hex(lpad(unhex(12345), 2, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex(''), 5, unhex('1f'))) +-- !query analysis +Project [hex(lpad(lpad, unhex(, false), 5, unhex(1f, false))) AS hex(lpad(unhex(), 5, unhex(1f)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aa'), 5, unhex('1f'))) +-- !query analysis +Project [hex(lpad(lpad, unhex(aa, false), 5, unhex(1f, false))) AS hex(lpad(unhex(aa), 5, unhex(1f)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aa'), 6, unhex('1f'))) +-- !query analysis +Project [hex(lpad(lpad, unhex(aa, false), 6, unhex(1f, false))) AS hex(lpad(unhex(aa), 6, unhex(1f)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex(''), 5, unhex('1f2e'))) +-- !query analysis +Project [hex(lpad(lpad, unhex(, false), 5, unhex(1f2e, false))) AS hex(lpad(unhex(), 5, unhex(1f2e)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aa'), 5, unhex('1f2e'))) +-- !query analysis +Project [hex(lpad(lpad, unhex(aa, false), 5, unhex(1f2e, false))) AS hex(lpad(unhex(aa), 5, unhex(1f2e)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aa'), 6, unhex('1f2e'))) +-- !query analysis +Project [hex(lpad(lpad, unhex(aa, false), 6, unhex(1f2e, false))) AS hex(lpad(unhex(aa), 6, unhex(1f2e)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex(''), 6, unhex(''))) +-- !query analysis +Project [hex(lpad(lpad, unhex(, false), 6, unhex(, false))) AS hex(lpad(unhex(), 6, unhex()))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aabbcc'), 6, unhex(''))) +-- !query analysis +Project [hex(lpad(lpad, unhex(aabbcc, false), 6, unhex(, false))) AS hex(lpad(unhex(aabbcc), 6, unhex()))#x] ++- OneRowRelation + + +-- !query +SELECT hex(lpad(unhex('aabbcc'), 2, unhex('ff'))) +-- !query analysis +Project [hex(lpad(lpad, unhex(aabbcc, false), 2, unhex(ff, false))) AS hex(lpad(unhex(aabbcc), 2, unhex(ff)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex(''), 5)) +-- !query analysis +Project [hex(rpad(rpad, unhex(, false), 5, 0x00)) AS hex(rpad(unhex(), 5, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aabb'), 5)) +-- !query analysis +Project [hex(rpad(rpad, unhex(aabb, false), 5, 0x00)) AS hex(rpad(unhex(aabb), 5, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aabbcc'), 2)) +-- !query analysis +Project [hex(rpad(rpad, unhex(aabbcc, false), 2, 0x00)) AS hex(rpad(unhex(aabbcc), 2, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('123'), 2)) +-- !query analysis +Project [hex(rpad(rpad, unhex(123, false), 2, 0x00)) AS hex(rpad(unhex(123), 2, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('12345'), 2)) +-- !query analysis +Project [hex(rpad(rpad, unhex(12345, false), 2, 0x00)) AS hex(rpad(unhex(12345), 2, X'00'))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex(''), 5, unhex('1f'))) +-- !query analysis +Project [hex(rpad(rpad, unhex(, false), 5, unhex(1f, false))) AS hex(rpad(unhex(), 5, unhex(1f)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aa'), 5, unhex('1f'))) +-- !query analysis +Project [hex(rpad(rpad, unhex(aa, false), 5, unhex(1f, false))) AS hex(rpad(unhex(aa), 5, unhex(1f)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aa'), 6, unhex('1f'))) +-- !query analysis +Project [hex(rpad(rpad, unhex(aa, false), 6, unhex(1f, false))) AS hex(rpad(unhex(aa), 6, unhex(1f)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex(''), 5, unhex('1f2e'))) +-- !query analysis +Project [hex(rpad(rpad, unhex(, false), 5, unhex(1f2e, false))) AS hex(rpad(unhex(), 5, unhex(1f2e)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aa'), 5, unhex('1f2e'))) +-- !query analysis +Project [hex(rpad(rpad, unhex(aa, false), 5, unhex(1f2e, false))) AS hex(rpad(unhex(aa), 5, unhex(1f2e)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aa'), 6, unhex('1f2e'))) +-- !query analysis +Project [hex(rpad(rpad, unhex(aa, false), 6, unhex(1f2e, false))) AS hex(rpad(unhex(aa), 6, unhex(1f2e)))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex(''), 6, unhex(''))) +-- !query analysis +Project [hex(rpad(rpad, unhex(, false), 6, unhex(, false))) AS hex(rpad(unhex(), 6, unhex()))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aabbcc'), 6, unhex(''))) +-- !query analysis +Project [hex(rpad(rpad, unhex(aabbcc, false), 6, unhex(, false))) AS hex(rpad(unhex(aabbcc), 6, unhex()))#x] ++- OneRowRelation + + +-- !query +SELECT hex(rpad(unhex('aabbcc'), 2, unhex('ff'))) +-- !query analysis +Project [hex(rpad(rpad, unhex(aabbcc, false), 2, unhex(ff, false))) AS hex(rpad(unhex(aabbcc), 2, unhex(ff)))#x] ++- OneRowRelation + + +-- !query +SELECT lpad('abc', 5, x'57') +-- !query analysis +Project [lpad(abc, 5, cast(0x57 as string)) AS lpad(abc, 5, X'57')#x] ++- OneRowRelation + + +-- !query +SELECT lpad(x'57', 5, 'abc') +-- !query analysis +Project [lpad(cast(0x57 as string), 5, abc) AS lpad(X'57', 5, abc)#x] ++- OneRowRelation + + +-- !query +SELECT rpad('abc', 5, x'57') +-- !query analysis +Project [rpad(abc, 5, cast(0x57 as string)) AS rpad(abc, 5, X'57')#x] ++- OneRowRelation + + +-- !query +SELECT rpad(x'57', 5, 'abc') +-- !query analysis +Project [rpad(cast(0x57 as string), 5, abc) AS rpad(X'57', 5, abc)#x] ++- OneRowRelation + + +-- !query +select decode() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "2", + "functionName" : "`decode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "decode()" + } ] +} + + +-- !query +select decode(encode('abc', 'utf-8')) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "2", + "functionName" : "`decode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "decode(encode('abc', 'utf-8'))" + } ] +} + + +-- !query +select decode(encode('abc', 'utf-8'), 'utf-8') +-- !query analysis +Project [decode(encode(abc, utf-8), utf-8) AS decode(encode(abc, utf-8), utf-8)#x] ++- OneRowRelation + + +-- !query +select decode(1, 1, 'Southlake') +-- !query analysis +Project [decode(1, 1, Southlake) AS decode(1, 1, Southlake)#x] ++- OneRowRelation + + +-- !query +select decode(2, 1, 'Southlake') +-- !query analysis +Project [decode(2, 1, Southlake) AS decode(2, 1, Southlake)#x] ++- OneRowRelation + + +-- !query +select decode(2, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle', 'Non domestic') +-- !query analysis +Project [decode(2, 1, Southlake, 2, San Francisco, 3, New Jersey, 4, Seattle, Non domestic) AS decode(2, 1, Southlake, 2, San Francisco, 3, New Jersey, 4, Seattle, Non domestic)#x] ++- OneRowRelation + + +-- !query +select decode(6, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle', 'Non domestic') +-- !query analysis +Project [decode(6, 1, Southlake, 2, San Francisco, 3, New Jersey, 4, Seattle, Non domestic) AS decode(6, 1, Southlake, 2, San Francisco, 3, New Jersey, 4, Seattle, Non domestic)#x] ++- OneRowRelation + + +-- !query +select decode(6, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle') +-- !query analysis +Project [decode(6, 1, Southlake, 2, San Francisco, 3, New Jersey, 4, Seattle) AS decode(6, 1, Southlake, 2, San Francisco, 3, New Jersey, 4, Seattle)#x] ++- OneRowRelation + + +-- !query +select decode(null, 6, 'Spark', NULL, 'SQL', 4, 'rocks') +-- !query analysis +Project [decode(null, 6, Spark, null, SQL, 4, rocks) AS decode(NULL, 6, Spark, NULL, SQL, 4, rocks)#x] ++- OneRowRelation + + +-- !query +select decode(null, 6, 'Spark', NULL, 'SQL', 4, 'rocks', NULL, '.') +-- !query analysis +Project [decode(null, 6, Spark, null, SQL, 4, rocks, null, .) AS decode(NULL, 6, Spark, NULL, SQL, 4, rocks, NULL, .)#x] ++- OneRowRelation + + +-- !query +SELECT CONTAINS(null, 'Spark') +-- !query analysis +Project [Contains(cast(null as string), Spark) AS contains(NULL, Spark)#x] ++- OneRowRelation + + +-- !query +SELECT CONTAINS('Spark SQL', null) +-- !query analysis +Project [Contains(Spark SQL, cast(null as string)) AS contains(Spark SQL, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT CONTAINS(null, null) +-- !query analysis +Project [Contains(cast(null as string), cast(null as string)) AS contains(NULL, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT CONTAINS('Spark SQL', 'Spark') +-- !query analysis +Project [Contains(Spark SQL, Spark) AS contains(Spark SQL, Spark)#x] ++- OneRowRelation + + +-- !query +SELECT CONTAINS('Spark SQL', 'SQL') +-- !query analysis +Project [Contains(Spark SQL, SQL) AS contains(Spark SQL, SQL)#x] ++- OneRowRelation + + +-- !query +SELECT CONTAINS('Spark SQL', 'SPARK') +-- !query analysis +Project [Contains(Spark SQL, SPARK) AS contains(Spark SQL, SPARK)#x] ++- OneRowRelation + + +-- !query +SELECT startswith('Spark SQL', 'ark') +-- !query analysis +Project [StartsWith(Spark SQL, ark) AS startswith(Spark SQL, ark)#x] ++- OneRowRelation + + +-- !query +SELECT startswith('Spark SQL', 'Spa') +-- !query analysis +Project [StartsWith(Spark SQL, Spa) AS startswith(Spark SQL, Spa)#x] ++- OneRowRelation + + +-- !query +SELECT startswith(null, 'Spark') +-- !query analysis +Project [StartsWith(cast(null as string), Spark) AS startswith(NULL, Spark)#x] ++- OneRowRelation + + +-- !query +SELECT startswith('Spark', null) +-- !query analysis +Project [StartsWith(Spark, cast(null as string)) AS startswith(Spark, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT startswith(null, null) +-- !query analysis +Project [StartsWith(cast(null as string), cast(null as string)) AS startswith(NULL, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT endswith('Spark SQL', 'QL') +-- !query analysis +Project [EndsWith(Spark SQL, QL) AS endswith(Spark SQL, QL)#x] ++- OneRowRelation + + +-- !query +SELECT endswith('Spark SQL', 'Spa') +-- !query analysis +Project [EndsWith(Spark SQL, Spa) AS endswith(Spark SQL, Spa)#x] ++- OneRowRelation + + +-- !query +SELECT endswith(null, 'Spark') +-- !query analysis +Project [EndsWith(cast(null as string), Spark) AS endswith(NULL, Spark)#x] ++- OneRowRelation + + +-- !query +SELECT endswith('Spark', null) +-- !query analysis +Project [EndsWith(Spark, cast(null as string)) AS endswith(Spark, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT endswith(null, null) +-- !query analysis +Project [EndsWith(cast(null as string), cast(null as string)) AS endswith(NULL, NULL)#x] ++- OneRowRelation + + +-- !query +SELECT contains(x'537061726b2053514c', x'537061726b') +-- !query analysis +Project [contains(contains, 0x537061726B2053514C, 0x537061726B) AS contains(X'537061726B2053514C', X'537061726B')#x] ++- OneRowRelation + + +-- !query +SELECT contains(x'', x'') +-- !query analysis +Project [contains(contains, 0x, 0x) AS contains(X'', X'')#x] ++- OneRowRelation + + +-- !query +SELECT contains(x'537061726b2053514c', null) +-- !query analysis +Project [Contains(cast(0x537061726B2053514C as string), cast(null as string)) AS contains(X'537061726B2053514C', NULL)#x] ++- OneRowRelation + + +-- !query +SELECT contains(12, '1') +-- !query analysis +Project [Contains(cast(12 as string), 1) AS contains(12, 1)#x] ++- OneRowRelation + + +-- !query +SELECT contains(true, 'ru') +-- !query analysis +Project [Contains(cast(true as string), ru) AS contains(true, ru)#x] ++- OneRowRelation + + +-- !query +SELECT contains(x'12', 12) +-- !query analysis +Project [Contains(cast(0x12 as string), cast(12 as string)) AS contains(X'12', 12)#x] ++- OneRowRelation + + +-- !query +SELECT contains(true, false) +-- !query analysis +Project [Contains(cast(true as string), cast(false as string)) AS contains(true, false)#x] ++- OneRowRelation + + +-- !query +SELECT startswith(x'537061726b2053514c', x'537061726b') +-- !query analysis +Project [startswith(startswith, 0x537061726B2053514C, 0x537061726B) AS startswith(X'537061726B2053514C', X'537061726B')#x] ++- OneRowRelation + + +-- !query +SELECT startswith(x'537061726b2053514c', x'') +-- !query analysis +Project [startswith(startswith, 0x537061726B2053514C, 0x) AS startswith(X'537061726B2053514C', X'')#x] ++- OneRowRelation + + +-- !query +SELECT startswith(x'', x'') +-- !query analysis +Project [startswith(startswith, 0x, 0x) AS startswith(X'', X'')#x] ++- OneRowRelation + + +-- !query +SELECT startswith(x'537061726b2053514c', null) +-- !query analysis +Project [StartsWith(cast(0x537061726B2053514C as string), cast(null as string)) AS startswith(X'537061726B2053514C', NULL)#x] ++- OneRowRelation + + +-- !query +SELECT endswith(x'537061726b2053514c', x'53516c') +-- !query analysis +Project [endswith(endswith, 0x537061726B2053514C, 0x53516C) AS endswith(X'537061726B2053514C', X'53516C')#x] ++- OneRowRelation + + +-- !query +SELECT endsWith(x'537061726b2053514c', x'537061726b') +-- !query analysis +Project [endswith(endswith, 0x537061726B2053514C, 0x537061726B) AS endswith(X'537061726B2053514C', X'537061726B')#x] ++- OneRowRelation + + +-- !query +SELECT endsWith(x'537061726b2053514c', x'') +-- !query analysis +Project [endswith(endswith, 0x537061726B2053514C, 0x) AS endswith(X'537061726B2053514C', X'')#x] ++- OneRowRelation + + +-- !query +SELECT endsWith(x'', x'') +-- !query analysis +Project [endswith(endswith, 0x, 0x) AS endswith(X'', X'')#x] ++- OneRowRelation + + +-- !query +SELECT endsWith(x'537061726b2053514c', null) +-- !query analysis +Project [EndsWith(cast(0x537061726B2053514C as string), cast(null as string)) AS endswith(X'537061726B2053514C', NULL)#x] ++- OneRowRelation + + +-- !query +select to_number('454', '000') +-- !query analysis +Project [to_number(454, 000) AS to_number(454, 000)#x] ++- OneRowRelation + + +-- !query +select to_number('454.2', '000.0') +-- !query analysis +Project [to_number(454.2, 000.0) AS to_number(454.2, 000.0)#x] ++- OneRowRelation + + +-- !query +select to_number('12,454', '00,000') +-- !query analysis +Project [to_number(12,454, 00,000) AS to_number(12,454, 00,000)#x] ++- OneRowRelation + + +-- !query +select to_number('$78.12', '$00.00') +-- !query analysis +Project [to_number($78.12, $00.00) AS to_number($78.12, $00.00)#x] ++- OneRowRelation + + +-- !query +select to_number('+454', 'S000') +-- !query analysis +Project [to_number(+454, S000) AS to_number(+454, S000)#x] ++- OneRowRelation + + +-- !query +select to_number('-454', 'S000') +-- !query analysis +Project [to_number(-454, S000) AS to_number(-454, S000)#x] ++- OneRowRelation + + +-- !query +select to_number('12,454.8-', '00,000.9MI') +-- !query analysis +Project [to_number(12,454.8-, 00,000.9MI) AS to_number(12,454.8-, 00,000.9MI)#x] ++- OneRowRelation + + +-- !query +select to_number('00,454.8-', '00,000.9MI') +-- !query analysis +Project [to_number(00,454.8-, 00,000.9MI) AS to_number(00,454.8-, 00,000.9MI)#x] ++- OneRowRelation + + +-- !query +select to_number('<00,454.8>', '00,000.9PR') +-- !query analysis +Project [to_number(<00,454.8>, 00,000.9PR) AS to_number(<00,454.8>, 00,000.9PR)#x] ++- OneRowRelation + + +-- !query +select to_binary('', 'base64') +-- !query analysis +Project [to_binary(, Some(base64), false) AS to_binary(, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary(' ', 'base64') +-- !query analysis +Project [to_binary( , Some(base64), false) AS to_binary( , base64)#x] ++- OneRowRelation + + +-- !query +select to_binary(' ab cd ', 'base64') +-- !query analysis +Project [to_binary( ab cd , Some(base64), false) AS to_binary( ab cd , base64)#x] ++- OneRowRelation + + +-- !query +select to_binary(' ab c=', 'base64') +-- !query analysis +Project [to_binary( ab c=, Some(base64), false) AS to_binary( ab c=, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary(' ab cdef= = ', 'base64') +-- !query analysis +Project [to_binary( ab cdef= = , Some(base64), false) AS to_binary( ab cdef= = , base64)#x] ++- OneRowRelation + + +-- !query +select to_binary( + concat(' b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0', + 'd2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl'), 'base64') +-- !query analysis +Project [to_binary(concat( b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0, d2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl), Some(base64), false) AS to_binary(concat( b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0, d2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl), base64)#x] ++- OneRowRelation + + +-- !query +select to_binary('a', 'base64') +-- !query analysis +Project [to_binary(a, Some(base64), false) AS to_binary(a, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary('a?', 'base64') +-- !query analysis +Project [to_binary(a?, Some(base64), false) AS to_binary(a?, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary('abcde', 'base64') +-- !query analysis +Project [to_binary(abcde, Some(base64), false) AS to_binary(abcde, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary('abcd=', 'base64') +-- !query analysis +Project [to_binary(abcd=, Some(base64), false) AS to_binary(abcd=, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary('a===', 'base64') +-- !query analysis +Project [to_binary(a===, Some(base64), false) AS to_binary(a===, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary('ab==f', 'base64') +-- !query analysis +Project [to_binary(ab==f, Some(base64), false) AS to_binary(ab==f, base64)#x] ++- OneRowRelation + + +-- !query +select to_binary( + '∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β)', 'utf-8') +-- !query analysis +Project [to_binary(∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β), Some(utf-8), false) AS to_binary(∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β), utf-8)#x] ++- OneRowRelation + + +-- !query +select to_binary('大千世界', 'utf8') +-- !query analysis +Project [to_binary(大千世界, Some(utf8), false) AS to_binary(大千世界, utf8)#x] ++- OneRowRelation + + +-- !query +select to_binary('', 'utf-8') +-- !query analysis +Project [to_binary(, Some(utf-8), false) AS to_binary(, utf-8)#x] ++- OneRowRelation + + +-- !query +select to_binary(' ', 'utf8') +-- !query analysis +Project [to_binary( , Some(utf8), false) AS to_binary( , utf8)#x] ++- OneRowRelation + + +-- !query +select to_binary('737472696E67') +-- !query analysis +Project [to_binary(737472696E67, None, false) AS to_binary(737472696E67)#x] ++- OneRowRelation + + +-- !query +select to_binary('737472696E67', 'hex') +-- !query analysis +Project [to_binary(737472696E67, Some(hex), false) AS to_binary(737472696E67, hex)#x] ++- OneRowRelation + + +-- !query +select to_binary('') +-- !query analysis +Project [to_binary(, None, false) AS to_binary()#x] ++- OneRowRelation + + +-- !query +select to_binary('1', 'hex') +-- !query analysis +Project [to_binary(1, Some(hex), false) AS to_binary(1, hex)#x] ++- OneRowRelation + + +-- !query +select to_binary('FF') +-- !query analysis +Project [to_binary(FF, None, false) AS to_binary(FF)#x] ++- OneRowRelation + + +-- !query +select to_binary('123', 'hex') +-- !query analysis +Project [to_binary(123, Some(hex), false) AS to_binary(123, hex)#x] ++- OneRowRelation + + +-- !query +select to_binary('12345', 'hex') +-- !query analysis +Project [to_binary(12345, Some(hex), false) AS to_binary(12345, hex)#x] ++- OneRowRelation + + +-- !query +select to_binary('GG') +-- !query analysis +Project [to_binary(GG, None, false) AS to_binary(GG)#x] ++- OneRowRelation + + +-- !query +select to_binary('01 AF', 'hex') +-- !query analysis +Project [to_binary(01 AF, Some(hex), false) AS to_binary(01 AF, hex)#x] ++- OneRowRelation + + +-- !query +select to_binary('abc', concat('utf', '-8')) +-- !query analysis +Project [to_binary(abc, Some(concat(utf, -8)), false) AS to_binary(abc, concat(utf, -8))#x] ++- OneRowRelation + + +-- !query +select to_binary(' ab cdef= = ', substr('base64whynot', 0, 6)) +-- !query analysis +Project [to_binary( ab cdef= = , Some(substr(base64whynot, 0, 6)), false) AS to_binary( ab cdef= = , substr(base64whynot, 0, 6))#x] ++- OneRowRelation + + +-- !query +select to_binary(' ab cdef= = ', replace('HEX0', '0')) +-- !query analysis +Project [to_binary( ab cdef= = , Some(replace(HEX0, 0, )), false) AS to_binary( ab cdef= = , replace(HEX0, 0, ))#x] ++- OneRowRelation + + +-- !query +select to_binary('abc', 'Hex') +-- !query analysis +Project [to_binary(abc, Some(Hex), false) AS to_binary(abc, Hex)#x] ++- OneRowRelation + + +-- !query +select to_binary('abc', null) +-- !query analysis +Project [to_binary(abc, Some(cast(null as string)), false) AS to_binary(abc, NULL)#x] ++- OneRowRelation + + +-- !query +select to_binary(null, 'utf-8') +-- !query analysis +Project [to_binary(cast(null as string), Some(utf-8), false) AS to_binary(NULL, utf-8)#x] ++- OneRowRelation + + +-- !query +select to_binary(null, null) +-- !query analysis +Project [to_binary(cast(null as string), Some(cast(null as string)), false) AS to_binary(NULL, NULL)#x] ++- OneRowRelation + + +-- !query +select to_binary(null, cast(null as string)) +-- !query analysis +Project [to_binary(cast(null as string), Some(cast(null as string)), false) AS to_binary(NULL, CAST(NULL AS STRING))#x] ++- OneRowRelation + + +-- !query +select to_binary('abc', 1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", + "sqlState" : "42K09", + "messageParameters" : { + "inputName" : "fmt", + "inputValue" : "'1'", + "requireType" : "case-insensitive \"STRING\"", + "sqlExpr" : "\"to_binary(abc, 1)\"", + "validValues" : "'hex', 'utf-8', 'utf8', or 'base64'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "to_binary('abc', 1)" + } ] +} + + +-- !query +select to_binary('abc', 'invalidFormat') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", + "sqlState" : "42K09", + "messageParameters" : { + "inputName" : "fmt", + "inputValue" : "'invalidformat'", + "requireType" : "case-insensitive \"STRING\"", + "sqlExpr" : "\"to_binary(abc, invalidFormat)\"", + "validValues" : "'hex', 'utf-8', 'utf8', or 'base64'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "to_binary('abc', 'invalidFormat')" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW fmtTable(fmtField) AS SELECT * FROM VALUES ('invalidFormat') +-- !query analysis +CreateViewCommand `fmtTable`, [(fmtField,None)], SELECT * FROM VALUES ('invalidFormat'), false, false, LocalTempView, true + +- Project [col1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT to_binary('abc', fmtField) FROM fmtTable +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"fmtField\"", + "inputName" : "fmt", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"to_binary(abc, fmtField)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "to_binary('abc', fmtField)" + } ] +} + + +-- !query +DROP VIEW IF EXISTS fmtTable +-- !query analysis +DropTempViewCommand fmtTable + + +-- !query +select luhn_check('4111111111111111') +-- !query analysis +Project [luhn_check(4111111111111111) AS luhn_check(4111111111111111)#x] ++- OneRowRelation + + +-- !query +select luhn_check('5500000000000004') +-- !query analysis +Project [luhn_check(5500000000000004) AS luhn_check(5500000000000004)#x] ++- OneRowRelation + + +-- !query +select luhn_check('340000000000009') +-- !query analysis +Project [luhn_check(340000000000009) AS luhn_check(340000000000009)#x] ++- OneRowRelation + + +-- !query +select luhn_check('6011000000000004') +-- !query analysis +Project [luhn_check(6011000000000004) AS luhn_check(6011000000000004)#x] ++- OneRowRelation + + +-- !query +select luhn_check('6011000000000005') +-- !query analysis +Project [luhn_check(6011000000000005) AS luhn_check(6011000000000005)#x] ++- OneRowRelation + + +-- !query +select luhn_check('378282246310006') +-- !query analysis +Project [luhn_check(378282246310006) AS luhn_check(378282246310006)#x] ++- OneRowRelation + + +-- !query +select luhn_check('0') +-- !query analysis +Project [luhn_check(0) AS luhn_check(0)#x] ++- OneRowRelation + + +-- !query +select luhn_check('4111111111111111 ') +-- !query analysis +Project [luhn_check(4111111111111111 ) AS luhn_check(4111111111111111 )#x] ++- OneRowRelation + + +-- !query +select luhn_check('4111111 111111111') +-- !query analysis +Project [luhn_check(4111111 111111111) AS luhn_check(4111111 111111111)#x] ++- OneRowRelation + + +-- !query +select luhn_check(' 4111111111111111') +-- !query analysis +Project [luhn_check( 4111111111111111) AS luhn_check( 4111111111111111)#x] ++- OneRowRelation + + +-- !query +select luhn_check('') +-- !query analysis +Project [luhn_check() AS luhn_check()#x] ++- OneRowRelation + + +-- !query +select luhn_check(' ') +-- !query analysis +Project [luhn_check( ) AS luhn_check( )#x] ++- OneRowRelation + + +-- !query +select luhn_check('510B105105105106') +-- !query analysis +Project [luhn_check(510B105105105106) AS luhn_check(510B105105105106)#x] ++- OneRowRelation + + +-- !query +select luhn_check('ABCDED') +-- !query analysis +Project [luhn_check(ABCDED) AS luhn_check(ABCDED)#x] ++- OneRowRelation + + +-- !query +select luhn_check(null) +-- !query analysis +Project [luhn_check(cast(null as string)) AS luhn_check(NULL)#x] ++- OneRowRelation + + +-- !query +select luhn_check(6011111111111117) +-- !query analysis +Project [luhn_check(cast(6011111111111117 as string)) AS luhn_check(6011111111111117)#x] ++- OneRowRelation + + +-- !query +select luhn_check(6011111111111118) +-- !query analysis +Project [luhn_check(cast(6011111111111118 as string)) AS luhn_check(6011111111111118)#x] ++- OneRowRelation + + +-- !query +select luhn_check(123.456) +-- !query analysis +Project [luhn_check(cast(123.456 as string)) AS luhn_check(123.456)#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/struct.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/struct.sql.out new file mode 100644 index 0000000000000..77633125692e8 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/struct.sql.out @@ -0,0 +1,95 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tbl_x AS VALUES + (1, NAMED_STRUCT('C', 'gamma', 'D', 'delta')), + (2, NAMED_STRUCT('C', 'epsilon', 'D', 'eta')), + (3, NAMED_STRUCT('C', 'theta', 'D', 'iota')) + AS T(ID, ST) +-- !query analysis +CreateViewCommand `tbl_x`, VALUES + (1, NAMED_STRUCT('C', 'gamma', 'D', 'delta')), + (2, NAMED_STRUCT('C', 'epsilon', 'D', 'eta')), + (3, NAMED_STRUCT('C', 'theta', 'D', 'iota')) + AS T(ID, ST), false, false, LocalTempView, true + +- SubqueryAlias T + +- LocalRelation [ID#x, ST#x] + + +-- !query +SELECT STRUCT('alpha', 'beta') ST +-- !query analysis +Project [struct(col1, alpha, col2, beta) AS ST#x] ++- OneRowRelation + + +-- !query +SELECT STRUCT('alpha' AS A, 'beta' AS B) ST +-- !query analysis +Project [struct(A, alpha, B, beta) AS ST#x] ++- OneRowRelation + + +-- !query +SELECT ID, STRUCT(ST.*) NST FROM tbl_x +-- !query analysis +Project [ID#x, struct(C, ST#x.C, D, ST#x.D) AS NST#x] ++- SubqueryAlias tbl_x + +- View (`tbl_x`, [ID#x,ST#x]) + +- Project [cast(ID#x as int) AS ID#x, cast(ST#x as struct) AS ST#x] + +- SubqueryAlias T + +- LocalRelation [ID#x, ST#x] + + +-- !query +SELECT ID, STRUCT(ST.*,CAST(ID AS STRING) AS E) NST FROM tbl_x +-- !query analysis +Project [ID#x, struct(C, ST#x.C, D, ST#x.D, E, cast(ID#x as string)) AS NST#x] ++- SubqueryAlias tbl_x + +- View (`tbl_x`, [ID#x,ST#x]) + +- Project [cast(ID#x as int) AS ID#x, cast(ST#x as struct) AS ST#x] + +- SubqueryAlias T + +- LocalRelation [ID#x, ST#x] + + +-- !query +SELECT ID, STRUCT(CAST(ID AS STRING) AS AA, ST.*) NST FROM tbl_x +-- !query analysis +Project [ID#x, struct(AA, cast(ID#x as string), C, ST#x.C, D, ST#x.D) AS NST#x] ++- SubqueryAlias tbl_x + +- View (`tbl_x`, [ID#x,ST#x]) + +- Project [cast(ID#x as int) AS ID#x, cast(ST#x as struct) AS ST#x] + +- SubqueryAlias T + +- LocalRelation [ID#x, ST#x] + + +-- !query +SELECT ID, STRUCT(ST.*).C NST FROM tbl_x +-- !query analysis +Project [ID#x, struct(C, ST#x.C, D, ST#x.D).C AS NST#x] ++- SubqueryAlias tbl_x + +- View (`tbl_x`, [ID#x,ST#x]) + +- Project [cast(ID#x as int) AS ID#x, cast(ST#x as struct) AS ST#x] + +- SubqueryAlias T + +- LocalRelation [ID#x, ST#x] + + +-- !query +SELECT ID, STRUCT(ST.C, ST.D).D NST FROM tbl_x +-- !query analysis +Project [ID#x, struct(C, ST#x.C, D, ST#x.D).D AS NST#x] ++- SubqueryAlias tbl_x + +- View (`tbl_x`, [ID#x,ST#x]) + +- Project [cast(ID#x as int) AS ID#x, cast(ST#x as struct) AS ST#x] + +- SubqueryAlias T + +- LocalRelation [ID#x, ST#x] + + +-- !query +SELECT ID, STRUCT(ST.C as STC, ST.D as STD).STD FROM tbl_x +-- !query analysis +Project [ID#x, struct(STC, ST#x.C, STD, ST#x.D).STD AS struct(ST.C AS STC, ST.D AS STD).STD#x] ++- SubqueryAlias tbl_x + +- View (`tbl_x`, [ID#x,ST#x]) + +- Project [cast(ID#x as int) AS ID#x, cast(ST#x as struct) AS ST#x] + +- SubqueryAlias T + +- LocalRelation [ID#x, ST#x] 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 new file mode 100644 index 0000000000000..5a7e5f8a4867a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subexp-elimination.sql.out @@ -0,0 +1,96 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +('{"a":1, "b":"2"}', '[{"a": 1, "b":2}, {"a":2, "b":2}]'), ('{"a":1, "b":"2"}', null), ('{"a":2, "b":"3"}', '[{"a": 3, "b":4}, {"a":4, "b":5}]'), ('{"a":5, "b":"6"}', '[{"a": 6, "b":7}, {"a":8, "b":9}]'), (null, '[{"a": 1, "b":2}, {"a":2, "b":2}]') +AS testData(a, b) +-- !query analysis +CreateViewCommand `testData`, SELECT * FROM VALUES +('{"a":1, "b":"2"}', '[{"a": 1, "b":2}, {"a":2, "b":2}]'), ('{"a":1, "b":"2"}', null), ('{"a":2, "b":"3"}', '[{"a": 3, "b":4}, {"a":4, "b":5}]'), ('{"a":5, "b":"6"}', '[{"a": 6, "b":7}, {"a":8, "b":9}]'), (null, '[{"a": 1, "b":2}, {"a":2, "b":2}]') +AS testData(a, b), false, true, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT from_json(a, 'struct').a, from_json(a, 'struct').b, from_json(b, 'array>')[0].a, from_json(b, 'array>')[0].b FROM testData +-- !query analysis +Project [from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles)).a AS from_json(a).a#x, from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles)).b AS from_json(a).b#x, from_json(ArrayType(StructType(StructField(a,IntegerType,true),StructField(b,IntegerType,true)),true), b#x, Some(America/Los_Angeles))[0].a AS from_json(b)[0].a#x, from_json(ArrayType(StructType(StructField(a,IntegerType,true),StructField(b,IntegerType,true)),true), b#x, Some(America/Los_Angeles))[0].b AS from_json(b)[0].b#x] ++- 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] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT if(from_json(a, 'struct').a > 1, from_json(b, 'array>')[0].a, from_json(b, 'array>')[0].a + 1) FROM testData +-- !query analysis +Project [if ((from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles)).a > 1)) from_json(ArrayType(StructType(StructField(a,IntegerType,true),StructField(b,IntegerType,true)),true), b#x, Some(America/Los_Angeles))[0].a else (from_json(ArrayType(StructType(StructField(a,IntegerType,true),StructField(b,IntegerType,true)),true), b#x, Some(America/Los_Angeles))[0].a + 1) AS (IF((from_json(a).a > 1), from_json(b)[0].a, (from_json(b)[0].a + 1)))#x] ++- 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] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT if(isnull(from_json(a, 'struct').a), from_json(b, 'array>')[0].b + 1, from_json(b, 'array>')[0].b) FROM testData +-- !query analysis +Project [if (isnull(from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles)).a)) (from_json(ArrayType(StructType(StructField(a,IntegerType,true),StructField(b,IntegerType,true)),true), b#x, Some(America/Los_Angeles))[0].b + 1) else from_json(ArrayType(StructType(StructField(a,IntegerType,true),StructField(b,IntegerType,true)),true), b#x, Some(America/Los_Angeles))[0].b AS (IF((from_json(a).a IS NULL), (from_json(b)[0].b + 1), from_json(b)[0].b))#x] ++- 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] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !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)).a > 5) THEN from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles)).b WHEN (from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles)).a > 4) THEN cast((cast(from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles)).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)).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] ++- 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] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT case when from_json(a, 'struct').a > 5 then from_json(b, 'array>')[0].b when from_json(a, 'struct').a > 4 then from_json(b, 'array>')[0].b + 1 else from_json(b, 'array>')[0].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)).a > 5) THEN from_json(ArrayType(StructType(StructField(a,IntegerType,true),StructField(b,IntegerType,true)),true), b#x, Some(America/Los_Angeles))[0].b WHEN (from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles)).a > 4) THEN (from_json(ArrayType(StructType(StructField(a,IntegerType,true),StructField(b,IntegerType,true)),true), b#x, Some(America/Los_Angeles))[0].b + 1) ELSE (from_json(ArrayType(StructType(StructField(a,IntegerType,true),StructField(b,IntegerType,true)),true), b#x, Some(America/Los_Angeles))[0].b + 2) END AS CASE WHEN (from_json(a).a > 5) THEN from_json(b)[0].b WHEN (from_json(a).a > 4) THEN (from_json(b)[0].b + 1) ELSE (from_json(b)[0].b + 2) END#x] ++- 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] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT from_json(a, 'struct').a + random() > 2, from_json(a, 'struct').b, from_json(b, 'array>')[0].a, from_json(b, 'array>')[0].b + + random() > 2 FROM testData +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT if(from_json(a, 'struct').a + random() > 5, from_json(b, 'array>')[0].a, from_json(b, 'array>')[0].a + 1) FROM testData +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT case when from_json(a, 'struct').a > 5 then from_json(a, 'struct').b + random() > 5 when from_json(a, 'struct').a > 4 then from_json(a, 'struct').b + 1 + random() > 2 else from_json(a, 'struct').b + 2 + random() > 5 end FROM testData +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +DROP VIEW IF EXISTS testData +-- !query analysis +DropTempViewCommand testData diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-aggregate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-aggregate.sql.out new file mode 100644 index 0000000000000..baeb88169d5f7 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-aggregate.sql.out @@ -0,0 +1,312 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query analysis +CreateViewCommand `EMP`, SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id), false, false, LocalTempView, true + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query analysis +CreateViewCommand `DEPT`, SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state), false, false, LocalTempView, true + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query analysis +CreateViewCommand `BONUS`, SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt), false, false, LocalTempView, true + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +SELECT emp.dept_id, + avg(salary), + sum(salary) +FROM emp +WHERE EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id) +GROUP BY dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, sum(salary#x) AS sum(salary)#x] ++- Filter exists#x [dept_id#x] + : +- Project [state#x] + : +- Filter (dept_id#x = outer(dept_id#x)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT emp_name +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id) +-- !query analysis +Project [emp_name#x] ++- Filter exists#x [dept_id#x] + : +- Aggregate [dept_id#x], [max(dept_id#x) AS a#x] + : +- Filter (dept_id#x = outer(dept_id#x)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT count(*) +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id) +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter exists#x [dept_id#x] + : +- Aggregate [dept_id#x], [max(dept_id#x) AS a#x] + : +- Filter (dept_id#x = outer(dept_id#x)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM bonus +WHERE EXISTS (SELECT 1 + FROM emp + WHERE emp.emp_name = bonus.emp_name + AND EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE emp.dept_id = dept.dept_id + GROUP BY dept.dept_id)) +-- !query analysis +Project [emp_name#x, bonus_amt#x] ++- Filter exists#x [emp_name#x] + : +- Project [1 AS 1#x] + : +- Filter ((emp_name#x = outer(emp_name#x)) AND exists#x [dept_id#x]) + : : +- Aggregate [dept_id#x], [max(dept_id#x) AS max(dept_id)#x] + : : +- Filter (outer(dept_id#x) = dept_id#x) + : : +- SubqueryAlias dept + : : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : : +- Project [dept_id#x, dept_name#x, state#x] + : : +- SubqueryAlias DEPT + : : +- LocalRelation [dept_id#x, dept_name#x, state#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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias bonus + +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +SELECT emp.dept_id, + Avg(salary), + Sum(salary) +FROM emp +WHERE NOT EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id) +GROUP BY dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, sum(salary#x) AS sum(salary)#x] ++- Filter NOT exists#x [dept_id#x] + : +- Project [state#x] + : +- Filter (dept_id#x = outer(dept_id#x)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT emp_name +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id) +-- !query analysis +Project [emp_name#x] ++- Filter NOT exists#x [dept_id#x] + : +- Aggregate [dept_id#x], [max(dept_id#x) AS a#x] + : +- Filter (dept_id#x = outer(dept_id#x)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT count(*) +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id) +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter NOT exists#x [dept_id#x] + : +- Aggregate [dept_id#x], [max(dept_id#x) AS a#x] + : +- Filter (dept_id#x = outer(dept_id#x)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM bonus +WHERE NOT EXISTS (SELECT 1 + FROM emp + WHERE emp.emp_name = bonus.emp_name + AND EXISTS (SELECT Max(dept.dept_id) + FROM dept + WHERE emp.dept_id = dept.dept_id + GROUP BY dept.dept_id)) +-- !query analysis +Project [emp_name#x, bonus_amt#x] ++- Filter NOT exists#x [emp_name#x] + : +- Project [1 AS 1#x] + : +- Filter ((emp_name#x = outer(emp_name#x)) AND exists#x [dept_id#x]) + : : +- Aggregate [dept_id#x], [max(dept_id#x) AS max(dept_id)#x] + : : +- Filter (outer(dept_id#x) = dept_id#x) + : : +- SubqueryAlias dept + : : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : : +- Project [dept_id#x, dept_name#x, state#x] + : : +- SubqueryAlias DEPT + : : +- LocalRelation [dept_id#x, dept_name#x, state#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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias bonus + +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-basic.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-basic.sql.out new file mode 100644 index 0000000000000..f219b87bc1e2f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-basic.sql.out @@ -0,0 +1,345 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query analysis +CreateViewCommand `EMP`, SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id), false, false, LocalTempView, true + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query analysis +CreateViewCommand `DEPT`, SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state), false, false, LocalTempView, true + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query analysis +CreateViewCommand `BONUS`, SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt), false, false, LocalTempView, true + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter exists#x [] + : +- Project [1 AS 1#x] + : +- Filter ((dept_id#x > 10) AND (dept_id#x < 30)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter exists#x [dept_id#x] + : +- Project [dept_name#x] + : +- Filter (outer(dept_id#x) = dept_id#x) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id + OR emp.dept_id IS NULL) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter exists#x [dept_id#x && dept_id#x] + : +- Project [dept_name#x] + : +- Filter ((outer(dept_id#x) = dept_id#x) OR isnull(outer(dept_id#x))) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) + AND emp.id > 200 +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter (exists#x [dept_id#x] AND (id#x > 200)) + : +- Project [dept_name#x] + : +- Filter (outer(dept_id#x) = dept_id#x) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id) + AND emp.id > 200 +-- !query analysis +Project [emp_name#x] ++- Filter (exists#x [dept_id#x] AND (id#x > 200)) + : +- Project [state#x] + : +- Filter (outer(dept_id#x) = dept_id#x) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM dept +WHERE NOT EXISTS (SELECT emp_name + FROM emp + WHERE emp.dept_id = dept.dept_id) +-- !query analysis +Project [dept_id#x, dept_name#x, state#x] ++- Filter NOT exists#x [dept_id#x] + : +- Project [emp_name#x] + : +- Filter (dept_id#x = outer(dept_id#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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias dept + +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +SELECT * +FROM dept +WHERE NOT EXISTS (SELECT emp_name + FROM emp + WHERE emp.dept_id = dept.dept_id + OR state = 'NJ') +-- !query analysis +Project [dept_id#x, dept_name#x, state#x] ++- Filter NOT exists#x [dept_id#x && state#x] + : +- Project [emp_name#x] + : +- Filter ((dept_id#x = outer(dept_id#x)) OR (outer(state#x) = NJ)) + : +- 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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias dept + +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +SELECT * +FROM bonus +WHERE NOT EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) +-- !query analysis +Project [emp_name#x, bonus_amt#x] ++- Filter NOT exists#x [bonus_amt#x] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- Filter ((emp_name#x = emp_name#x) AND (outer(bonus_amt#x) > salary#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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias bonus + +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +SELECT emp.* +FROM emp +WHERE NOT EXISTS (SELECT NULL + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter NOT exists#x [emp_name#x] + : +- Project [null AS NULL#x] + : +- Filter (emp_name#x = outer(emp_name#x)) + : +- SubqueryAlias bonus + : +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + : +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + : +- Project [emp_name#x, bonus_amt#x] + : +- SubqueryAlias BONUS + : +- LocalRelation [emp_name#x, bonus_amt#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM bonus +WHERE EXISTS (SELECT emp_name + FROM emp + WHERE bonus.emp_name = emp.emp_name + AND EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id)) +-- !query analysis +Project [emp_name#x, bonus_amt#x] ++- Filter exists#x [emp_name#x] + : +- Project [emp_name#x] + : +- Filter ((outer(emp_name#x) = emp_name#x) AND exists#x [dept_id#x]) + : : +- Project [state#x] + : : +- Filter (dept_id#x = outer(dept_id#x)) + : : +- SubqueryAlias dept + : : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : : +- Project [dept_id#x, dept_name#x, state#x] + : : +- SubqueryAlias DEPT + : : +- LocalRelation [dept_id#x, dept_name#x, state#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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias bonus + +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out new file mode 100644 index 0000000000000..2cd6ba5356371 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out @@ -0,0 +1,384 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query analysis +CreateViewCommand `EMP`, SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id), false, false, LocalTempView, true + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query analysis +CreateViewCommand `DEPT`, SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state), false, false, LocalTempView, true + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query analysis +CreateViewCommand `BONUS`, SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt), false, false, LocalTempView, true + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +WITH bonus_cte + AS (SELECT * + FROM bonus + WHERE EXISTS (SELECT dept.dept_id, + emp.emp_name, + Max(salary), + Count(*) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY dept.dept_id, + emp.emp_name + ORDER BY emp.emp_name)) +SELECT * +FROM bonus a +WHERE a.bonus_amt > 30 + AND EXISTS (SELECT 1 + FROM bonus_cte b + WHERE a.emp_name = b.emp_name) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias bonus_cte +: +- Project [emp_name#x, bonus_amt#x] +: +- Filter exists#x [emp_name#x] +: : +- Sort [emp_name#x ASC NULLS FIRST], true +: : +- Aggregate [dept_id#x, emp_name#x], [dept_id#x, emp_name#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL] +: : +- Filter (outer(emp_name#x) = emp_name#x) +: : +- Join Inner, (dept_id#x = dept_id#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] +: : : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] +: : : +- SubqueryAlias EMP +: : : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] +: : +- SubqueryAlias dept +: : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: : +- Project [dept_id#x, dept_name#x, state#x] +: : +- SubqueryAlias DEPT +: : +- LocalRelation [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias bonus +: +- View (`BONUS`, [emp_name#x,bonus_amt#x]) +: +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] +: +- Project [emp_name#x, bonus_amt#x] +: +- SubqueryAlias BONUS +: +- LocalRelation [emp_name#x, bonus_amt#x] ++- Project [emp_name#x, bonus_amt#x] + +- Filter ((bonus_amt#x > cast(30 as double)) AND exists#x [emp_name#x]) + : +- Project [1 AS 1#x] + : +- Filter (outer(emp_name#x) = emp_name#x) + : +- SubqueryAlias b + : +- SubqueryAlias bonus_cte + : +- CTERelationRef xxxx, true, [emp_name#x, bonus_amt#x] + +- SubqueryAlias a + +- SubqueryAlias bonus + +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +WITH emp_cte + AS (SELECT * + FROM emp + WHERE id >= 100 + AND id <= 300), + dept_cte + AS (SELECT * + FROM dept + WHERE dept_id = 10) +SELECT * +FROM bonus +WHERE EXISTS (SELECT * + FROM emp_cte a + JOIN dept_cte b + ON a.dept_id = b.dept_id + WHERE bonus.emp_name = a.emp_name) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias emp_cte +: +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] +: +- Filter ((id#x >= 100) AND (id#x <= 300)) +: +- 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] +: +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] +: +- SubqueryAlias EMP +: +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias dept_cte +: +- Project [dept_id#x, dept_name#x, state#x] +: +- Filter (dept_id#x = 10) +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#x] ++- Project [emp_name#x, bonus_amt#x] + +- Filter exists#x [emp_name#x] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x, dept_id#x, dept_name#x, state#x] + : +- Filter (outer(emp_name#x) = emp_name#x) + : +- Join Inner, (dept_id#x = dept_id#x) + : :- SubqueryAlias a + : : +- SubqueryAlias emp_cte + : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias b + : +- SubqueryAlias dept_cte + : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias bonus + +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +WITH emp_cte + AS (SELECT * + FROM emp + WHERE id >= 100 + AND id <= 300), + dept_cte + AS (SELECT * + FROM dept + WHERE dept_id = 10) +SELECT DISTINCT b.emp_name, + b.bonus_amt +FROM bonus b, + emp_cte e, + dept d +WHERE e.dept_id = d.dept_id + AND e.emp_name = b.emp_name + AND EXISTS (SELECT * + FROM emp_cte a + LEFT JOIN dept_cte b + ON a.dept_id = b.dept_id + WHERE e.emp_name = a.emp_name) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias emp_cte +: +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] +: +- Filter ((id#x >= 100) AND (id#x <= 300)) +: +- 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] +: +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] +: +- SubqueryAlias EMP +: +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias dept_cte +: +- Project [dept_id#x, dept_name#x, state#x] +: +- Filter (dept_id#x = 10) +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#x] ++- Distinct + +- Project [emp_name#x, bonus_amt#x] + +- Filter (((dept_id#x = dept_id#x) AND (emp_name#x = emp_name#x)) AND exists#x [emp_name#x]) + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x, dept_id#x, dept_name#x, state#x] + : +- Filter (outer(emp_name#x) = emp_name#x) + : +- Join LeftOuter, (dept_id#x = dept_id#x) + : :- SubqueryAlias a + : : +- SubqueryAlias emp_cte + : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias b + : +- SubqueryAlias dept_cte + : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x] + +- Join Inner + :- Join Inner + : :- SubqueryAlias b + : : +- SubqueryAlias bonus + : : +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + : : +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + : : +- Project [emp_name#x, bonus_amt#x] + : : +- SubqueryAlias BONUS + : : +- LocalRelation [emp_name#x, bonus_amt#x] + : +- SubqueryAlias e + : +- SubqueryAlias emp_cte + : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias d + +- SubqueryAlias dept + +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +WITH empdept + AS (SELECT id, + salary, + emp_name, + dept.dept_id + FROM emp + LEFT JOIN dept + ON emp.dept_id = dept.dept_id + WHERE emp.id IN ( 100, 200 )) +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE EXISTS (SELECT dept_id, + max(salary) + FROM empdept + GROUP BY dept_id + HAVING count(*) > 1) +GROUP BY emp_name +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias empdept +: +- Project [id#x, salary#x, emp_name#x, dept_id#x] +: +- Filter id#x IN (100,200) +: +- Join LeftOuter, (dept_id#x = dept_id#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] +: : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] +: : +- SubqueryAlias EMP +: : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#x] ++- Aggregate [emp_name#x], [emp_name#x, sum(bonus_amt#x) AS sum(bonus_amt)#x] + +- Filter exists#x [] + : +- Project [dept_id#x, max(salary)#x] + : +- Filter (count(1)#xL > cast(1 as bigint)) + : +- Aggregate [dept_id#x], [dept_id#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL] + : +- SubqueryAlias empdept + : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x] + +- SubqueryAlias bonus + +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +WITH empdept + AS (SELECT id, + salary, + emp_name, + dept.dept_id + FROM emp + LEFT JOIN dept + ON emp.dept_id = dept.dept_id + WHERE emp.id IN ( 100, 200 )) +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE NOT EXISTS (SELECT dept_id, + Max(salary) + FROM empdept + GROUP BY dept_id + HAVING count(*) < 1) +GROUP BY emp_name +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias empdept +: +- Project [id#x, salary#x, emp_name#x, dept_id#x] +: +- Filter id#x IN (100,200) +: +- Join LeftOuter, (dept_id#x = dept_id#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] +: : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] +: : +- SubqueryAlias EMP +: : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#x] ++- Aggregate [emp_name#x], [emp_name#x, sum(bonus_amt#x) AS sum(bonus_amt)#x] + +- Filter NOT exists#x [] + : +- Project [dept_id#x, max(salary)#x] + : +- Filter (count(1)#xL < cast(1 as bigint)) + : +- Aggregate [dept_id#x], [dept_id#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL] + : +- SubqueryAlias empdept + : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x] + +- SubqueryAlias bonus + +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-having.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-having.sql.out new file mode 100644 index 0000000000000..da242e583f45e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-having.sql.out @@ -0,0 +1,264 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query analysis +CreateViewCommand `EMP`, SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id), false, false, LocalTempView, true + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query analysis +CreateViewCommand `DEPT`, SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state), false, false, LocalTempView, true + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query analysis +CreateViewCommand `BONUS`, SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt), false, false, LocalTempView, true + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +SELECT dept_id, count(*) +FROM emp +GROUP BY dept_id +HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt < min(emp.salary)) +-- !query analysis +Project [dept_id#x, count(1)#xL] ++- Filter exists#x [min(salary#x)#x] + : +- Project [1 AS 1#x] + : +- Filter (bonus_amt#x < outer(min(salary#x)#x)) + : +- SubqueryAlias bonus + : +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + : +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + : +- Project [emp_name#x, bonus_amt#x] + : +- SubqueryAlias BONUS + : +- LocalRelation [emp_name#x, bonus_amt#x] + +- Aggregate [dept_id#x], [dept_id#x, count(1) AS count(1)#xL, min(salary#x) AS min(salary#x)#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM dept +WHERE EXISTS (SELECT dept_id, + Count(*) + FROM emp + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt < Min(emp.salary))) +-- !query analysis +Project [dept_id#x, dept_name#x, state#x] ++- Filter exists#x [] + : +- Project [dept_id#x, count(1)#xL] + : +- Filter exists#x [min(salary#x)#x] + : : +- Project [1 AS 1#x] + : : +- Filter (bonus_amt#x < outer(min(salary#x)#x)) + : : +- SubqueryAlias bonus + : : +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + : : +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + : : +- Project [emp_name#x, bonus_amt#x] + : : +- SubqueryAlias BONUS + : : +- LocalRelation [emp_name#x, bonus_amt#x] + : +- Aggregate [dept_id#x], [dept_id#x, count(1) AS count(1)#xL, min(salary#x) AS min(salary#x)#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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias dept + +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +SELECT dept_id, + Max(salary) +FROM emp gp +WHERE EXISTS (SELECT dept_id, + Count(*) + FROM emp p + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt < Min(p.salary))) +GROUP BY gp.dept_id +-- !query analysis +Aggregate [dept_id#x], [dept_id#x, max(salary#x) AS max(salary)#x] ++- Filter exists#x [] + : +- Project [dept_id#x, count(1)#xL] + : +- Filter exists#x [min(salary#x)#x] + : : +- Project [1 AS 1#x] + : : +- Filter (bonus_amt#x < outer(min(salary#x)#x)) + : : +- SubqueryAlias bonus + : : +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + : : +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + : : +- Project [emp_name#x, bonus_amt#x] + : : +- SubqueryAlias BONUS + : : +- LocalRelation [emp_name#x, bonus_amt#x] + : +- Aggregate [dept_id#x], [dept_id#x, count(1) AS count(1)#xL, min(salary#x) AS min(salary#x)#x] + : +- SubqueryAlias p + : +- 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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias gp + +- 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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM dept +WHERE EXISTS (SELECT dept_id, + Count(*) + FROM emp + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt > Min(emp.salary))) +-- !query analysis +Project [dept_id#x, dept_name#x, state#x] ++- Filter exists#x [] + : +- Project [dept_id#x, count(1)#xL] + : +- Filter exists#x [min(salary#x)#x] + : : +- Project [1 AS 1#x] + : : +- Filter (bonus_amt#x > outer(min(salary#x)#x)) + : : +- SubqueryAlias bonus + : : +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + : : +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + : : +- Project [emp_name#x, bonus_amt#x] + : : +- SubqueryAlias BONUS + : : +- LocalRelation [emp_name#x, bonus_amt#x] + : +- Aggregate [dept_id#x], [dept_id#x, count(1) AS count(1)#xL, min(salary#x) AS min(salary#x)#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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias dept + +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +SELECT * +FROM dept +WHERE EXISTS (SELECT dept_id, + count(emp.dept_id) + FROM emp + WHERE dept.dept_id = dept_id + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE ( bonus_amt > min(emp.salary) + AND count(emp.dept_id) > 1 ))) +-- !query analysis +Project [dept_id#x, dept_name#x, state#x] ++- Filter exists#x [dept_id#x] + : +- Project [dept_id#x, count(dept_id)#xL] + : +- Filter exists#x [min(salary#x)#x && count(dept_id)#xL] + : : +- Project [1 AS 1#x] + : : +- Filter ((bonus_amt#x > outer(min(salary#x)#x)) AND (outer(count(dept_id)#xL) > cast(1 as bigint))) + : : +- SubqueryAlias bonus + : : +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + : : +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + : : +- Project [emp_name#x, bonus_amt#x] + : : +- SubqueryAlias BONUS + : : +- LocalRelation [emp_name#x, bonus_amt#x] + : +- Aggregate [dept_id#x], [dept_id#x, count(dept_id#x) AS count(dept_id)#xL, min(salary#x) AS min(salary#x)#x] + : +- Filter (outer(dept_id#x) = dept_id#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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias dept + +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out new file mode 100644 index 0000000000000..8ea5bf7baf1c2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out @@ -0,0 +1,898 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query analysis +CreateViewCommand `EMP`, SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id), false, false, LocalTempView, true + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query analysis +CreateViewCommand `DEPT`, SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state), false, false, LocalTempView, true + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query analysis +CreateViewCommand `BONUS`, SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt), false, false, LocalTempView, true + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +SELECT * +FROM emp, + dept +WHERE emp.dept_id = dept.dept_id + AND EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x, dept_id#x, dept_name#x, state#x] ++- Filter ((dept_id#x = dept_id#x) AND exists#x [emp_name#x]) + : +- Project [emp_name#x, bonus_amt#x] + : +- Filter (emp_name#x = outer(emp_name#x)) + : +- SubqueryAlias bonus + : +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + : +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + : +- Project [emp_name#x, bonus_amt#x] + : +- SubqueryAlias BONUS + : +- LocalRelation [emp_name#x, bonus_amt#x] + +- Join Inner + :- 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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias dept + +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +SELECT * +FROM emp + JOIN dept + ON emp.dept_id = dept.dept_id +WHERE EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x, dept_id#x, dept_name#x, state#x] ++- Filter exists#x [emp_name#x] + : +- Project [emp_name#x, bonus_amt#x] + : +- Filter (emp_name#x = outer(emp_name#x)) + : +- SubqueryAlias bonus + : +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + : +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + : +- Project [emp_name#x, bonus_amt#x] + : +- SubqueryAlias BONUS + : +- LocalRelation [emp_name#x, bonus_amt#x] + +- Join Inner, (dept_id#x = dept_id#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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias dept + +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +SELECT * +FROM emp + LEFT JOIN dept + ON emp.dept_id = dept.dept_id +WHERE EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x, dept_id#x, dept_name#x, state#x] ++- Filter exists#x [emp_name#x] + : +- Project [emp_name#x, bonus_amt#x] + : +- Filter (emp_name#x = outer(emp_name#x)) + : +- SubqueryAlias bonus + : +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + : +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + : +- Project [emp_name#x, bonus_amt#x] + : +- SubqueryAlias BONUS + : +- LocalRelation [emp_name#x, bonus_amt#x] + +- Join LeftOuter, (dept_id#x = dept_id#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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias dept + +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +SELECT * +FROM emp, + dept +WHERE emp.dept_id = dept.dept_id + AND NOT EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x, dept_id#x, dept_name#x, state#x] ++- Filter ((dept_id#x = dept_id#x) AND NOT exists#x [emp_name#x]) + : +- Project [emp_name#x, bonus_amt#x] + : +- Filter (emp_name#x = outer(emp_name#x)) + : +- SubqueryAlias bonus + : +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + : +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + : +- Project [emp_name#x, bonus_amt#x] + : +- SubqueryAlias BONUS + : +- LocalRelation [emp_name#x, bonus_amt#x] + +- Join Inner + :- 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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias dept + +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +SELECT * +FROM bonus +WHERE EXISTS (SELECT * + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name) +-- !query analysis +Project [emp_name#x, bonus_amt#x] ++- Filter exists#x [emp_name#x] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x, dept_id#x, dept_name#x, state#x] + : +- Filter (outer(emp_name#x) = emp_name#x) + : +- Join Inner, (dept_id#x = dept_id#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] + : : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : : +- SubqueryAlias EMP + : : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias bonus + +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +SELECT * +FROM bonus +WHERE EXISTS (SELECT * + FROM emp + RIGHT JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name) +-- !query analysis +Project [emp_name#x, bonus_amt#x] ++- Filter exists#x [emp_name#x] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x, dept_id#x, dept_name#x, state#x] + : +- Filter (outer(emp_name#x) = emp_name#x) + : +- Join RightOuter, (dept_id#x = dept_id#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] + : : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : : +- SubqueryAlias EMP + : : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias bonus + +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +SELECT * +FROM bonus +WHERE EXISTS (SELECT dept.dept_id, + emp.emp_name, + Max(salary), + Count(*) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY dept.dept_id, + emp.emp_name + ORDER BY emp.emp_name) +-- !query analysis +Project [emp_name#x, bonus_amt#x] ++- Filter exists#x [emp_name#x] + : +- Sort [emp_name#x ASC NULLS FIRST], true + : +- Aggregate [dept_id#x, emp_name#x], [dept_id#x, emp_name#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL] + : +- Filter (outer(emp_name#x) = emp_name#x) + : +- Join Inner, (dept_id#x = dept_id#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] + : : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : : +- SubqueryAlias EMP + : : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias bonus + +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE EXISTS (SELECT emp_name, + Max(salary) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY emp_name + HAVING Count(*) > 1 + ORDER BY emp_name) +GROUP BY emp_name +-- !query analysis +Aggregate [emp_name#x], [emp_name#x, sum(bonus_amt#x) AS sum(bonus_amt)#x] ++- Filter exists#x [emp_name#x] + : +- Sort [emp_name#x ASC NULLS FIRST], true + : +- Project [emp_name#x, max(salary)#x] + : +- Filter (count(1)#xL > cast(1 as bigint)) + : +- Aggregate [emp_name#x], [emp_name#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL] + : +- Filter (outer(emp_name#x) = emp_name#x) + : +- Join Inner, (dept_id#x = dept_id#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] + : : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : : +- SubqueryAlias EMP + : : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias bonus + +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE NOT EXISTS (SELECT emp_name, + Max(salary) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY emp_name + HAVING Count(*) > 1 + ORDER BY emp_name) +GROUP BY emp_name +-- !query analysis +Aggregate [emp_name#x], [emp_name#x, sum(bonus_amt#x) AS sum(bonus_amt)#x] ++- Filter NOT exists#x [emp_name#x] + : +- Sort [emp_name#x ASC NULLS FIRST], true + : +- Project [emp_name#x, max(salary)#x] + : +- Filter (count(1)#xL > cast(1 as bigint)) + : +- Aggregate [emp_name#x], [emp_name#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL] + : +- Filter (outer(emp_name#x) = emp_name#x) + : +- Join Inner, (dept_id#x = dept_id#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] + : : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : : +- SubqueryAlias EMP + : : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias bonus + +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + UNION + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter exists#x [] + : +- Distinct + : +- Union false, false + : :- Project [dept_id#x, dept_name#x, state#x] + : : +- Filter (dept_id#x < 30) + : : +- SubqueryAlias dept + : : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : : +- Project [dept_id#x, dept_name#x, state#x] + : : +- SubqueryAlias DEPT + : : +- LocalRelation [dept_id#x, dept_name#x, state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- Filter ((dept_id#x >= 30) AND (dept_id#x <= 50)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter exists#x [] + : +- Intersect false + : :- Project [dept_id#x, dept_name#x, state#x] + : : +- Filter (dept_id#x < 30) + : : +- SubqueryAlias dept + : : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : : +- Project [dept_id#x, dept_name#x, state#x] + : : +- SubqueryAlias DEPT + : : +- LocalRelation [dept_id#x, dept_name#x, state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- Filter ((dept_id#x >= 30) AND (dept_id#x <= 50)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter NOT exists#x [] + : +- Intersect false + : :- Project [dept_id#x, dept_name#x, state#x] + : : +- Filter (dept_id#x < 30) + : : +- SubqueryAlias dept + : : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : : +- Project [dept_id#x, dept_name#x, state#x] + : : +- SubqueryAlias DEPT + : : +- LocalRelation [dept_id#x, dept_name#x, state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- Filter ((dept_id#x >= 30) AND (dept_id#x <= 50)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + EXCEPT + SELECT * + FROM dept + WHERE dept_id > 50) +UNION ALL +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50) +-- !query analysis +Union false, false +:- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] +: +- Filter exists#x [] +: : +- Except false +: : :- Project [dept_id#x, dept_name#x, state#x] +: : : +- SubqueryAlias dept +: : : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: : : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: : : +- Project [dept_id#x, dept_name#x, state#x] +: : : +- SubqueryAlias DEPT +: : : +- LocalRelation [dept_id#x, dept_name#x, state#x] +: : +- Project [dept_id#x, dept_name#x, state#x] +: : +- Filter (dept_id#x > 50) +: : +- SubqueryAlias dept +: : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: : +- Project [dept_id#x, dept_name#x, state#x] +: : +- SubqueryAlias DEPT +: : +- LocalRelation [dept_id#x, dept_name#x, state#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] +: +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] +: +- SubqueryAlias EMP +: +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- Filter exists#x [] + : +- Intersect false + : :- Project [dept_id#x, dept_name#x, state#x] + : : +- Filter (dept_id#x < 30) + : : +- SubqueryAlias dept + : : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : : +- Project [dept_id#x, dept_name#x, state#x] + : : +- SubqueryAlias DEPT + : : +- LocalRelation [dept_id#x, dept_name#x, state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- Filter ((dept_id#x >= 30) AND (dept_id#x <= 50)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + EXCEPT + SELECT * + FROM dept + WHERE dept_id > 50) +UNION +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50) +-- !query analysis +Distinct ++- Union false, false + :- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- Filter exists#x [] + : : +- Except false + : : :- Project [dept_id#x, dept_name#x, state#x] + : : : +- SubqueryAlias dept + : : : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : : : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : : : +- Project [dept_id#x, dept_name#x, state#x] + : : : +- SubqueryAlias DEPT + : : : +- LocalRelation [dept_id#x, dept_name#x, state#x] + : : +- Project [dept_id#x, dept_name#x, state#x] + : : +- Filter (dept_id#x > 50) + : : +- SubqueryAlias dept + : : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : : +- Project [dept_id#x, dept_name#x, state#x] + : : +- SubqueryAlias DEPT + : : +- LocalRelation [dept_id#x, dept_name#x, state#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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- Filter exists#x [] + : +- Intersect false + : :- Project [dept_id#x, dept_name#x, state#x] + : : +- Filter (dept_id#x < 30) + : : +- SubqueryAlias dept + : : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : : +- Project [dept_id#x, dept_name#x, state#x] + : : +- SubqueryAlias DEPT + : : +- LocalRelation [dept_id#x, dept_name#x, state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- Filter ((dept_id#x >= 30) AND (dept_id#x <= 50)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + UNION + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA))\n+- SubqueryAlias dept\n +- View (`DEPT`, [dept_id#x,dept_name#x,state#x])\n +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x]\n +- Project [dept_id#x, dept_name#x, state#x]\n +- SubqueryAlias DEPT\n +- LocalRelation [dept_id#x, dept_name#x, state#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 131, + "fragment" : "SELECT *\n FROM dept\n WHERE dept_id = emp.dept_id and state = \"CA\"" + } ] +} + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + UNION + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA))\n+- SubqueryAlias dept\n +- View (`DEPT`, [dept_id#x,dept_name#x,state#x])\n +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x]\n +- Project [dept_id#x, dept_name#x, state#x]\n +- SubqueryAlias DEPT\n +- LocalRelation [dept_id#x, dept_name#x, state#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 134, + "fragment" : "SELECT *\n FROM dept\n WHERE dept_id = emp.dept_id and state = \"CA\"" + } ] +} + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + INTERSECT ALL + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA))\n+- SubqueryAlias dept\n +- View (`DEPT`, [dept_id#x,dept_name#x,state#x])\n +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x]\n +- Project [dept_id#x, dept_name#x, state#x]\n +- SubqueryAlias DEPT\n +- LocalRelation [dept_id#x, dept_name#x, state#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 131, + "fragment" : "SELECT *\n FROM dept\n WHERE dept_id = emp.dept_id and state = \"CA\"" + } ] +} + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + INTERSECT DISTINCT + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA))\n+- SubqueryAlias dept\n +- View (`DEPT`, [dept_id#x,dept_name#x,state#x])\n +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x]\n +- Project [dept_id#x, dept_name#x, state#x]\n +- SubqueryAlias DEPT\n +- LocalRelation [dept_id#x, dept_name#x, state#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 35, + "stopIndex" : 130, + "fragment" : "SELECT *\n FROM dept\n WHERE dept_id = emp.dept_id and state = \"CA\"" + } ] +} + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + EXCEPT ALL + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA))\n+- SubqueryAlias dept\n +- View (`DEPT`, [dept_id#x,dept_name#x,state#x])\n +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x]\n +- Project [dept_id#x, dept_name#x, state#x]\n +- SubqueryAlias DEPT\n +- LocalRelation [dept_id#x, dept_name#x, state#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 131, + "fragment" : "SELECT *\n FROM dept\n WHERE dept_id = emp.dept_id and state = \"CA\"" + } ] +} + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + EXCEPT DISTINCT + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA))\n+- SubqueryAlias dept\n +- View (`DEPT`, [dept_id#x,dept_name#x,state#x])\n +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x]\n +- Project [dept_id#x, dept_name#x, state#x]\n +- SubqueryAlias DEPT\n +- LocalRelation [dept_id#x, dept_name#x, state#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 131, + "fragment" : "SELECT *\n FROM dept\n WHERE dept_id = emp.dept_id and state = \"CA\"" + } ] +} + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + INTERSECT ALL + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA))\n+- SubqueryAlias dept\n +- View (`DEPT`, [dept_id#x,dept_name#x,state#x])\n +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x]\n +- Project [dept_id#x, dept_name#x, state#x]\n +- SubqueryAlias DEPT\n +- LocalRelation [dept_id#x, dept_name#x, state#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 134, + "fragment" : "SELECT *\n FROM dept\n WHERE dept_id = emp.dept_id and state = \"CA\"" + } ] +} + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + EXCEPT DISTINCT + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter ((dept_id#x = outer(dept_id#x)) AND (state#x = CA))\n+- SubqueryAlias dept\n +- View (`DEPT`, [dept_id#x,dept_name#x,state#x])\n +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x]\n +- Project [dept_id#x, dept_name#x, state#x]\n +- SubqueryAlias DEPT\n +- LocalRelation [dept_id#x, dept_name#x, state#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 134, + "fragment" : "SELECT *\n FROM dept\n WHERE dept_id = emp.dept_id and state = \"CA\"" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-orderby-limit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-orderby-limit.sql.out new file mode 100644 index 0000000000000..c688eb202574d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-orderby-limit.sql.out @@ -0,0 +1,567 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query analysis +CreateViewCommand `EMP`, SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id), false, false, LocalTempView, true + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query analysis +CreateViewCommand `DEPT`, SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state), false, false, LocalTempView, true + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query analysis +CreateViewCommand `BONUS`, SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt), false, false, LocalTempView, true + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state) +ORDER BY hiredate +-- !query analysis +Sort [hiredate#x ASC NULLS FIRST], true ++- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- Filter exists#x [dept_id#x] + : +- Project [dept_id#x] + : +- Sort [state#x ASC NULLS FIRST], true + : +- Project [dept_id#x, state#x] + : +- Filter (outer(dept_id#x) = dept_id#x) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT id, + hiredate +FROM emp +WHERE EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state) +ORDER BY hiredate DESC +-- !query analysis +Sort [hiredate#x DESC NULLS LAST], true ++- Project [id#x, hiredate#x] + +- Filter exists#x [dept_id#x] + : +- Project [dept_id#x] + : +- Sort [state#x ASC NULLS FIRST], true + : +- Project [dept_id#x, state#x] + : +- Filter (outer(dept_id#x) = dept_id#x) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state) +ORDER BY hiredate +-- !query analysis +Sort [hiredate#x ASC NULLS FIRST], true ++- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- Filter NOT exists#x [dept_id#x] + : +- Project [dept_id#x] + : +- Sort [state#x ASC NULLS FIRST], true + : +- Project [dept_id#x, state#x] + : +- Filter (outer(dept_id#x) = dept_id#x) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT emp_name +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY state + ORDER BY state) +-- !query analysis +Project [emp_name#x] ++- Filter NOT exists#x [dept_id#x] + : +- Project [a#x] + : +- Sort [state#x ASC NULLS FIRST], true + : +- Aggregate [state#x], [max(dept_id#x) AS a#x, state#x] + : +- Filter (dept_id#x = outer(dept_id#x)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT count(*) +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept_id + ORDER BY dept_id) +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter NOT exists#x [dept_id#x] + : +- Project [a#x] + : +- Sort [dept_id#x ASC NULLS FIRST], true + : +- Aggregate [dept_id#x], [max(dept_id#x) AS a#x, dept_id#x] + : +- Filter (dept_id#x = outer(dept_id#x)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 10 + LIMIT 1) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter exists#x [] + : +- GlobalLimit 1 + : +- LocalLimit 1 + : +- Project [dept_name#x] + : +- Filter (dept_id#x > 10) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + GROUP BY state + LIMIT 1) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter exists#x [] + : +- GlobalLimit 1 + : +- LocalLimit 1 + : +- Aggregate [state#x], [max(dept_id#x) AS max(dept_id)#x] + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 100 + LIMIT 1) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter NOT exists#x [] + : +- GlobalLimit 1 + : +- LocalLimit 1 + : +- Project [dept_name#x] + : +- Filter (dept_id#x > 100) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id > 100 + GROUP BY state + LIMIT 1) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter NOT exists#x [] + : +- GlobalLimit 1 + : +- LocalLimit 1 + : +- Aggregate [state#x], [max(dept_id#x) AS max(dept_id)#x] + : +- Filter (dept_id#x > 100) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 10 + LIMIT 1 + OFFSET 2) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter exists#x [] + : +- GlobalLimit 1 + : +- LocalLimit 1 + : +- Offset 2 + : +- Project [dept_name#x] + : +- Filter (dept_id#x > 10) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + GROUP BY state + LIMIT 1 + OFFSET 2) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter exists#x [] + : +- GlobalLimit 1 + : +- LocalLimit 1 + : +- Offset 2 + : +- Aggregate [state#x], [max(dept_id#x) AS max(dept_id)#x] + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 100 + LIMIT 1 + OFFSET 2) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter NOT exists#x [] + : +- GlobalLimit 1 + : +- LocalLimit 1 + : +- Offset 2 + : +- Project [dept_name#x] + : +- Filter (dept_id#x > 100) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id > 100 + GROUP BY state + LIMIT 1 + OFFSET 2) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter NOT exists#x [] + : +- GlobalLimit 1 + : +- LocalLimit 1 + : +- Offset 2 + : +- Aggregate [state#x], [max(dept_id#x) AS max(dept_id)#x] + : +- Filter (dept_id#x > 100) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 10 + OFFSET 2) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter exists#x [] + : +- Offset 2 + : +- Project [dept_name#x] + : +- Filter (dept_id#x > 10) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + GROUP BY state + OFFSET 2) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter exists#x [] + : +- Offset 2 + : +- Aggregate [state#x], [max(dept_id#x) AS max(dept_id)#x] + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 100 + OFFSET 2) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter NOT exists#x [] + : +- Offset 2 + : +- Project [dept_name#x] + : +- Filter (dept_id#x > 100) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id > 100 + GROUP BY state + OFFSET 2) +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter NOT exists#x [] + : +- Offset 2 + : +- Aggregate [state#x], [max(dept_id#x) AS max(dept_id)#x] + : +- Filter (dept_id#x > 100) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-outside-filter.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-outside-filter.sql.out new file mode 100644 index 0000000000000..0813f3612dd1d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-outside-filter.sql.out @@ -0,0 +1,422 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query analysis +CreateViewCommand `EMP`, SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id), false, false, LocalTempView, true + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query analysis +CreateViewCommand `DEPT`, SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state), false, false, LocalTempView, true + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query analysis +CreateViewCommand `BONUS`, SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt), false, false, LocalTempView, true + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +SELECT + emp_name, + EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30) +FROM emp +-- !query analysis +Project [emp_name#x, exists#x [] AS exists()#x] +: +- Project [1 AS 1#x] +: +- Filter ((dept_id#x > 10) AND (dept_id#x < 30)) +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT + emp_name, + EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) +FROM emp +-- !query analysis +Project [emp_name#x, exists#x [dept_id#x] AS exists(dept_id)#x] +: +- Project [1 AS 1#x] +: +- Filter (outer(dept_id#x) = dept_id#x) +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT + sum(salary), + sum(salary) FILTER (WHERE EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30)) +FROM emp +-- !query analysis +Aggregate [sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE exists#x []) AS sum(salary) FILTER (WHERE exists())#x] +: +- Project [1 AS 1#x] +: +- Filter ((dept_id#x > 10) AND (dept_id#x < 30)) +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT + sum(salary), + sum(salary) FILTER (WHERE EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id)) +FROM emp +-- !query analysis +Aggregate [sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE exists#x [dept_id#x]) AS sum(salary) FILTER (WHERE exists(dept_id))#x] +: +- Project [1 AS 1#x] +: +- Filter (outer(dept_id#x) = dept_id#x) +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT + sum(salary), + sum(salary) FILTER (WHERE EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR EXISTS (SELECT 1 + FROM bonus + WHERE emp.emp_name = bonus.emp_name)) +FROM emp +-- !query analysis +Aggregate [sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE (exists#x [dept_id#x] OR exists#x [emp_name#x])) AS sum(salary) FILTER (WHERE (exists(dept_id) OR exists(emp_name)))#x] +: :- Project [1 AS 1#x] +: : +- Filter (outer(dept_id#x) = dept_id#x) +: : +- SubqueryAlias dept +: : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: : +- Project [dept_id#x, dept_name#x, state#x] +: : +- SubqueryAlias DEPT +: : +- LocalRelation [dept_id#x, dept_name#x, state#x] +: +- Project [1 AS 1#x] +: +- Filter (outer(emp_name#x) = emp_name#x) +: +- SubqueryAlias bonus +: +- View (`BONUS`, [emp_name#x,bonus_amt#x]) +: +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] +: +- Project [emp_name#x, bonus_amt#x] +: +- SubqueryAlias BONUS +: +- LocalRelation [emp_name#x, bonus_amt#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT + sum(DISTINCT salary), + count(DISTINCT hiredate) FILTER (WHERE EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id)) +FROM emp +-- !query analysis +Aggregate [sum(distinct salary#x) AS sum(DISTINCT salary)#x, count(distinct hiredate#x) FILTER (WHERE exists#x [dept_id#x]) AS count(DISTINCT hiredate) FILTER (WHERE exists(dept_id))#xL] +: +- Project [1 AS 1#x] +: +- Filter (outer(dept_id#x) = dept_id#x) +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT + count(hiredate), + sum(salary) +FROM emp +GROUP BY EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) +-- !query analysis +Aggregate [exists#x [dept_id#x]], [count(hiredate#x) AS count(hiredate)#xL, sum(salary#x) AS sum(salary)#x] +: +- Project [1 AS 1#x] +: +- Filter (outer(dept_id#x) = dept_id#x) +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT + count(DISTINCT hiredate), + sum(DISTINCT salary) +FROM emp +GROUP BY EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) +-- !query analysis +Aggregate [exists#x [dept_id#x]], [count(distinct hiredate#x) AS count(DISTINCT hiredate)#xL, sum(distinct salary#x) AS sum(DISTINCT salary)#x] +: +- Project [1 AS 1#x] +: +- Filter (outer(dept_id#x) = dept_id#x) +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT + count(CASE WHEN EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30) THEN 1 END), + sum(CASE WHEN EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30) THEN salary END) +FROM emp +-- !query analysis +Aggregate [count(CASE WHEN exists#x [] THEN 1 END) AS count(CASE WHEN exists() THEN 1 END)#xL, sum(CASE WHEN exists#x [] THEN salary#x END) AS sum(CASE WHEN exists() THEN salary END)#x] +: :- Project [1 AS 1#x] +: : +- Filter ((dept_id#x > 10) AND (dept_id#x < 30)) +: : +- SubqueryAlias dept +: : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: : +- Project [dept_id#x, dept_name#x, state#x] +: : +- SubqueryAlias DEPT +: : +- LocalRelation [dept_id#x, dept_name#x, state#x] +: +- Project [1 AS 1#x] +: +- Filter ((dept_id#x > 10) AND (dept_id#x < 30)) +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT + count(CASE WHEN EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) THEN 1 END), + sum(CASE WHEN EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) THEN salary END) +FROM emp +-- !query analysis +Aggregate [count(CASE WHEN exists#x [dept_id#x] THEN 1 END) AS count(CASE WHEN exists(dept_id) THEN 1 END)#xL, sum(CASE WHEN exists#x [dept_id#x] THEN salary#x END) AS sum(CASE WHEN exists(dept_id) THEN salary END)#x] +: :- Project [1 AS 1#x] +: : +- Filter (outer(dept_id#x) = dept_id#x) +: : +- SubqueryAlias dept +: : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: : +- Project [dept_id#x, dept_name#x, state#x] +: : +- SubqueryAlias DEPT +: : +- LocalRelation [dept_id#x, dept_name#x, state#x] +: +- Project [1 AS 1#x] +: +- Filter (outer(dept_id#x) = dept_id#x) +: +- SubqueryAlias dept +: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) +: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] +: +- Project [dept_id#x, dept_name#x, state#x] +: +- SubqueryAlias DEPT +: +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT + emp_name, + sum(salary) OVER (PARTITION BY EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30)) +FROM emp +-- !query analysis +Project [emp_name#x, sum(salary) OVER (PARTITION BY exists() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] ++- Project [emp_name#x, salary#x, _w1#x, sum(salary) OVER (PARTITION BY exists() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, sum(salary) OVER (PARTITION BY exists() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Window [sum(salary#x) windowspecdefinition(_w1#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS sum(salary) OVER (PARTITION BY exists() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x], [_w1#x] + +- Project [emp_name#x, salary#x, exists#x [] AS _w1#x] + : +- Project [1 AS 1#x] + : +- Filter ((dept_id#x > 10) AND (dept_id#x < 30)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT + emp_name, + sum(salary) OVER (PARTITION BY EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id)) +FROM emp +-- !query analysis +Project [emp_name#x, sum(salary) OVER (PARTITION BY exists(dept_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] ++- Project [emp_name#x, salary#x, _w1#x, sum(salary) OVER (PARTITION BY exists(dept_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, sum(salary) OVER (PARTITION BY exists(dept_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Window [sum(salary#x) windowspecdefinition(_w1#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS sum(salary) OVER (PARTITION BY exists(dept_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x], [_w1#x] + +- Project [emp_name#x, salary#x, exists#x [dept_id#x] AS _w1#x] + : +- Project [1 AS 1#x] + : +- Filter (outer(dept_id#x) = dept_id#x) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-within-and-or.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-within-and-or.sql.out new file mode 100644 index 0000000000000..427c1ccf7f74a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-within-and-or.sql.out @@ -0,0 +1,246 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query analysis +CreateViewCommand `EMP`, SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id), false, false, LocalTempView, true + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query analysis +CreateViewCommand `DEPT`, SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state), false, false, LocalTempView, true + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query analysis +CreateViewCommand `BONUS`, SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt), false, false, LocalTempView, true + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR emp.id > 200 +-- !query analysis +Project [emp_name#x] ++- Filter (exists#x [dept_id#x] OR (id#x > 200)) + : +- Project [state#x] + : +- Filter (outer(dept_id#x) = dept_id#x) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR emp.dept_id IS NULL +-- !query analysis +Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ++- Filter (exists#x [dept_id#x] OR isnull(dept_id#x)) + : +- Project [dept_name#x] + : +- Filter (outer(dept_id#x) = dept_id#x) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id + AND dept.dept_id = 20) + OR EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id + AND dept.dept_id = 30) +-- !query analysis +Project [emp_name#x] ++- Filter (exists#x [dept_id#x] OR exists#x [dept_id#x]) + : :- Project [state#x] + : : +- Filter ((outer(dept_id#x) = dept_id#x) AND (dept_id#x = 20)) + : : +- SubqueryAlias dept + : : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : : +- Project [dept_id#x, dept_name#x, state#x] + : : +- SubqueryAlias DEPT + : : +- LocalRelation [dept_id#x, dept_name#x, state#x] + : +- Project [state#x] + : +- Filter ((outer(dept_id#x) = dept_id#x) AND (dept_id#x = 30)) + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#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] + +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + + +-- !query +SELECT * +FROM bonus +WHERE ( NOT EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) + OR EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + OR bonus_amt < emp.salary) ) +-- !query analysis +Project [emp_name#x, bonus_amt#x] ++- Filter (NOT exists#x [bonus_amt#x] OR exists#x [bonus_amt#x]) + : :- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : : +- Filter ((emp_name#x = emp_name#x) AND (outer(bonus_amt#x) > salary#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] + : : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : : +- SubqueryAlias EMP + : : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- Filter ((emp_name#x = emp_name#x) OR (outer(bonus_amt#x) < salary#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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias bonus + +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +SELECT * FROM bonus WHERE NOT EXISTS +( + SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) +AND +emp_name IN +( + SELECT emp_name + FROM emp + WHERE bonus_amt < emp.salary) +-- !query analysis +Project [emp_name#x, bonus_amt#x] ++- Filter (NOT exists#x [bonus_amt#x] AND emp_name#x IN (list#x [bonus_amt#x])) + : :- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : : +- Filter ((emp_name#x = emp_name#x) AND (outer(bonus_amt#x) > salary#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] + : : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : : +- SubqueryAlias EMP + : : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- Project [emp_name#x] + : +- Filter (outer(bonus_amt#x) < salary#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] + : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + +- SubqueryAlias bonus + +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-basic.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-basic.sql.out new file mode 100644 index 0000000000000..8541900a0eae4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-basic.sql.out @@ -0,0 +1,111 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view tab_a as select * from values (1, 1) as tab_a(a1, b1) +-- !query analysis +CreateViewCommand `tab_a`, select * from values (1, 1) as tab_a(a1, b1), false, false, LocalTempView, true + +- Project [a1#x, b1#x] + +- SubqueryAlias tab_a + +- LocalRelation [a1#x, b1#x] + + +-- !query +create temporary view tab_b as select * from values (1, 1) as tab_b(a2, b2) +-- !query analysis +CreateViewCommand `tab_b`, select * from values (1, 1) as tab_b(a2, b2), false, false, LocalTempView, true + +- Project [a2#x, b2#x] + +- SubqueryAlias tab_b + +- LocalRelation [a2#x, b2#x] + + +-- !query +create temporary view struct_tab as select struct(col1 as a, col2 as b) as record from + values (1, 1), (1, 2), (2, 1), (2, 2) +-- !query analysis +CreateViewCommand `struct_tab`, select struct(col1 as a, col2 as b) as record from + values (1, 1), (1, 2), (2, 1), (2, 2), false, false, LocalTempView, true + +- Project [struct(a, col1#x, b, col2#x) AS record#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select 1 from tab_a where (a1, b1) not in (select a2, b2 from tab_b) +-- !query analysis +Project [1 AS 1#x] ++- Filter NOT named_struct(a1, a1#x, b1, b1#x) IN (list#x []) + : +- Project [a2#x, b2#x] + : +- SubqueryAlias tab_b + : +- View (`tab_b`, [a2#x,b2#x]) + : +- Project [cast(a2#x as int) AS a2#x, cast(b2#x as int) AS b2#x] + : +- Project [a2#x, b2#x] + : +- SubqueryAlias tab_b + : +- LocalRelation [a2#x, b2#x] + +- SubqueryAlias tab_a + +- View (`tab_a`, [a1#x,b1#x]) + +- Project [cast(a1#x as int) AS a1#x, cast(b1#x as int) AS b1#x] + +- Project [a1#x, b1#x] + +- SubqueryAlias tab_a + +- LocalRelation [a1#x, b1#x] + + +-- !query +select 1 from tab_a where (a1, b1) not in (select (a2, b2) from tab_b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_LENGTH_MISMATCH", + "sqlState" : "42K09", + "messageParameters" : { + "leftColumns" : "\"a1\", \"b1\"", + "leftLength" : "2", + "rightColumns" : "\"named_struct(a2, a2, b2, b2)\"", + "rightLength" : "1", + "sqlExpr" : "\"(named_struct('a1', a1, 'b1', b1) IN (listquery()))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 70, + "fragment" : "not in (select (a2, b2) from tab_b)" + } ] +} + + +-- !query +select count(*) from struct_tab where record in + (select (a2 as a, b2 as b) from tab_b) +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter record#x IN (list#x []) + : +- Project [named_struct(a, a2#x, b, b2#x) AS named_struct(a, a2 AS a, b, b2 AS b)#x] + : +- SubqueryAlias tab_b + : +- View (`tab_b`, [a2#x,b2#x]) + : +- Project [cast(a2#x as int) AS a2#x, cast(b2#x as int) AS b2#x] + : +- Project [a2#x, b2#x] + : +- SubqueryAlias tab_b + : +- LocalRelation [a2#x, b2#x] + +- SubqueryAlias struct_tab + +- View (`struct_tab`, [record#x]) + +- Project [cast(record#x as struct) AS record#x] + +- Project [struct(a, col1#x, b, col2#x) AS record#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select count(*) from struct_tab where record not in + (select (a2 as a, b2 as b) from tab_b) +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter NOT record#x IN (list#x []) + : +- Project [named_struct(a, a2#x, b, b2#x) AS named_struct(a, a2 AS a, b, b2 AS b)#x] + : +- SubqueryAlias tab_b + : +- View (`tab_b`, [a2#x,b2#x]) + : +- Project [cast(a2#x as int) AS a2#x, cast(b2#x as int) AS b2#x] + : +- Project [a2#x, b2#x] + : +- SubqueryAlias tab_b + : +- LocalRelation [a2#x, b2#x] + +- SubqueryAlias struct_tab + +- View (`struct_tab`, [record#x]) + +- Project [cast(record#x as struct) AS record#x] + +- Project [struct(a, col1#x, b, col2#x) AS record#x] + +- LocalRelation [col1#x, col2#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-group-by.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-group-by.sql.out new file mode 100644 index 0000000000000..11c596eee1cb2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-group-by.sql.out @@ -0,0 +1,641 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query analysis +CreateViewCommand `t1`, select * from values + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +create temporary view t2 as select * from values + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query analysis +CreateViewCommand `t2`, select * from values + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i), false, false, LocalTempView, true + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +create temporary view t3 as select * from values + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query analysis +CreateViewCommand `t3`, select * from values + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i), false, false, LocalTempView, true + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT t1a, + Avg(t1b) +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2) +GROUP BY t1a +-- !query analysis +Aggregate [t1a#x], [t1a#x, avg(t1b#x) AS avg(t1b)#x] ++- Filter t1a#x IN (list#x []) + : +- Project [t2a#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + Max(t1b) +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +GROUP BY t1a, + t1d +-- !query analysis +Aggregate [t1a#x, t1d#xL], [t1a#x, max(t1b#x) AS max(t1b)#x] ++- Filter t1b#x IN (list#x [t1a#x]) + : +- Project [t2b#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) +GROUP BY t1a, + t1b +-- !query analysis +Aggregate [t1a#x, t1b#x], [t1a#x, t1b#x] ++- Filter t1c#x IN (list#x [t1a#x]) + : +- Project [t2c#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) + OR t1c IN (SELECT t3c + FROM t3 + WHERE t1a = t3a) +GROUP BY t1a, + t1c +-- !query analysis +Aggregate [t1a#x, t1c#x], [t1a#x, sum(distinct t1b#x) AS sum(DISTINCT t1b)#xL] ++- Filter (t1c#x IN (list#x [t1a#x]) OR t1c#x IN (list#x [t1a#x])) + : :- Project [t2c#x] + : : +- Filter (outer(t1a#x) = t2a#x) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Project [t3c#x] + : +- Filter (outer(t1a#x) = t3a#x) + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) + AND t1c IN (SELECT t3c + FROM t3 + WHERE t1a = t3a) +GROUP BY t1a, + t1c +-- !query analysis +Aggregate [t1a#x, t1c#x], [t1a#x, sum(distinct t1b#x) AS sum(DISTINCT t1b)#xL] ++- Filter (t1c#x IN (list#x [t1a#x]) AND t1c#x IN (list#x [t1a#x])) + : :- Project [t2c#x] + : : +- Filter (outer(t1a#x) = t2a#x) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Project [t3c#x] + : +- Filter (outer(t1a#x) = t3a#x) + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + Count(DISTINCT( t1b )) +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) +GROUP BY t1a, + t1c +HAVING t1a = "t1b" +-- !query analysis +Filter (t1a#x = t1b) ++- Aggregate [t1a#x, t1c#x], [t1a#x, count(distinct t1b#x) AS count(DISTINCT t1b)#xL] + +- Filter t1c#x IN (list#x [t1a#x]) + : +- Project [t2c#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1b IN (SELECT Max(t2b) + FROM t2 + GROUP BY t2a) +-- !query analysis +Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Filter t1b#x IN (list#x []) + : +- Aggregate [t2a#x], [max(t2b#x) AS max(t2b)#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM (SELECT t2a, + t2b + FROM t2 + WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b) + GROUP BY t2a, + t2b) t2 +-- !query analysis +Project [t2a#x, t2b#x] ++- SubqueryAlias t2 + +- Aggregate [t2a#x, t2b#x], [t2a#x, t2b#x] + +- Filter t2a#x IN (list#x [t2b#x]) + : +- Project [t1a#x] + : +- Filter (t1b#x = outer(t2b#x)) + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t2 + +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +SELECT Count(DISTINCT( * )) +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + AND t1c = t2c + GROUP BY t2a) +-- !query analysis +Aggregate [count(distinct t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x) AS count(DISTINCT t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i)#xL] ++- Filter t1b#x IN (list#x [t1a#x && t1c#x]) + : +- Aggregate [t2a#x], [min(t2b#x) AS min(t2b)#x] + : +- Filter ((outer(t1a#x) = t2a#x) AND (outer(t1c#x) = t2c#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT Max(t2c) + FROM t2 + WHERE t1a = t2a + GROUP BY t2a, + t2c + HAVING t2c > 8) +-- !query analysis +Project [t1a#x, t1b#x] ++- Filter t1c#x IN (list#x [t1a#x]) + : +- Project [max(t2c)#x] + : +- Filter (t2c#x > 8) + : +- Aggregate [t2a#x, t2c#x], [max(t2c#x) AS max(t2c)#x, t2c#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2a IN (SELECT Min(t3a) + FROM t3 + WHERE t3a = t2a + GROUP BY t3b) + GROUP BY t2c) +-- !query analysis +Project [t1a#x, t1b#x] ++- Filter t1c#x IN (list#x []) + : +- Aggregate [t2c#x], [t2c#x] + : +- Filter t2a#x IN (list#x [t2a#x]) + : : +- Aggregate [t3b#x], [min(t3a#x) AS min(t3a)#x] + : : +- Filter (t3a#x = outer(t2a#x)) + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a) +GROUP BY t1a +-- !query analysis +Aggregate [t1a#x], [t1a#x, min(t1b#x) AS min(t1b)#x] ++- Filter t1c#x IN (list#x [t1b#x]) + : +- Aggregate [t2a#x], [min(t2c#x) AS min(t2c)#x] + : +- Filter (t2b#x = outer(t1b#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b IN (SELECT Min(t3b) + FROM t3 + WHERE t2a = t3a + GROUP BY t3a) + GROUP BY t2c) +GROUP BY t1a, + t1d +-- !query analysis +Aggregate [t1a#x, t1d#xL], [t1a#x, min(t1b#x) AS min(t1b)#x] ++- Filter t1c#x IN (list#x []) + : +- Aggregate [t2c#x], [min(t2c#x) AS min(t2c)#x] + : +- Filter t2b#x IN (list#x [t2a#x]) + : : +- Aggregate [t3a#x], [min(t3b#x) AS min(t3b)#x] + : : +- Filter (outer(t2a#x) = t3a#x) + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a) + AND t1d IN (SELECT t3d + FROM t3 + WHERE t1c = t3c + GROUP BY t3d) +GROUP BY t1a +-- !query analysis +Aggregate [t1a#x], [t1a#x, min(t1b#x) AS min(t1b)#x] ++- Filter (t1c#x IN (list#x [t1b#x]) AND t1d#xL IN (list#x [t1c#x])) + : :- Aggregate [t2a#x], [min(t2c#x) AS min(t2c)#x] + : : +- Filter (t2b#x = outer(t1b#x)) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Aggregate [t3d#xL], [t3d#xL] + : +- Filter (outer(t1c#x) = t3c#x) + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a) + OR t1d IN (SELECT t3d + FROM t3 + WHERE t1c = t3c + GROUP BY t3d) +GROUP BY t1a +-- !query analysis +Aggregate [t1a#x], [t1a#x, min(t1b#x) AS min(t1b)#x] ++- Filter (t1c#x IN (list#x [t1b#x]) OR t1d#xL IN (list#x [t1c#x])) + : :- Aggregate [t2a#x], [min(t2c#x) AS min(t2c)#x] + : : +- Filter (t2b#x = outer(t1b#x)) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Aggregate [t3d#xL], [t3d#xL] + : +- Filter (outer(t1c#x) = t3c#x) + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a + HAVING t2a > t1a) + OR t1d IN (SELECT t3d + FROM t3 + WHERE t1c = t3c + GROUP BY t3d + HAVING t3d = t1d) +GROUP BY t1a +HAVING Min(t1b) IS NOT NULL +-- !query analysis +Filter isnotnull(min(t1b)#x) ++- Aggregate [t1a#x], [t1a#x, min(t1b#x) AS min(t1b)#x] + +- Filter (t1c#x IN (list#x [t1a#x && t1b#x]) OR t1d#xL IN (list#x [t1d#xL && t1c#x])) + : :- Project [min(t2c)#x] + : : +- Filter (t2a#x > outer(t1a#x)) + : : +- Aggregate [t2a#x], [min(t2c#x) AS min(t2c)#x, t2a#x] + : : +- Filter (t2b#x = outer(t1b#x)) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Filter (t3d#xL = outer(t1d#xL)) + : +- Aggregate [t3d#xL], [t3d#xL] + : +- Filter (outer(t1c#x) = t3c#x) + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-having.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-having.sql.out new file mode 100644 index 0000000000000..7f4935c8c79f6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-having.sql.out @@ -0,0 +1,387 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query analysis +CreateViewCommand `t1`, select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query analysis +CreateViewCommand `t2`, select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i), false, false, LocalTempView, true + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query analysis +CreateViewCommand `t3`, select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i), false, false, LocalTempView, true + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT t1a, + t1b, + t1h +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + GROUP BY t2b + HAVING t2b < 10) +-- !query analysis +Project [t1a#x, t1b#x, t1h#x] ++- Filter t1b#x IN (list#x []) + : +- Filter (cast(t2b#x as int) < 10) + : +- Aggregate [t2b#x], [t2b#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2b + HAVING t2b > 1) +-- !query analysis +Project [t1a#x, t1b#x, t1c#x] ++- Filter t1b#x IN (list#x [t1a#x]) + : +- Project [min(t2b)#x] + : +- Filter (cast(t2b#x as int) > 1) + : +- Aggregate [t2b#x], [min(t2b#x) AS min(t2b)#x, t2b#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, t1b, t1c +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1c < t2c) +GROUP BY t1a, t1b, t1c +HAVING t1b < 10 +-- !query analysis +Filter (cast(t1b#x as int) < 10) ++- Aggregate [t1a#x, t1b#x, t1c#x], [t1a#x, t1b#x, t1c#x] + +- Filter t1b#x IN (list#x [t1c#x]) + : +- Project [t2b#x] + : +- Filter (outer(t1c#x) < t2c#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, t1b, t1c +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1c = t2c) +GROUP BY t1a, t1b, t1c +HAVING COUNT (DISTINCT t1b) < 10 +-- !query analysis +Project [t1a#x, t1b#x, t1c#x] ++- Filter (count(distinct t1b#x)#xL < cast(10 as bigint)) + +- Aggregate [t1a#x, t1b#x, t1c#x], [t1a#x, t1b#x, t1c#x, count(distinct t1b#x) AS count(distinct t1b#x)#xL] + +- Filter t1b#x IN (list#x [t1c#x]) + : +- Project [t2b#x] + : +- Filter (outer(t1c#x) = t2c#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a + GROUP BY t2c + HAVING t2c > 10) +GROUP BY t1b +HAVING t1b >= 8 +-- !query analysis +Filter (cast(t1b#x as int) >= 8) ++- Aggregate [t1b#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x] + +- Filter t1c#x IN (list#x [t1a#x]) + : +- Filter (t2c#x > 10) + : +- Aggregate [t2c#x], [t2c#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + Max(t1b) +FROM t1 +WHERE t1b > 0 +GROUP BY t1a +HAVING t1a IN (SELECT t2a + FROM t2 + WHERE t2b IN (SELECT t3b + FROM t3 + WHERE t2c = t3c) + ) +-- !query analysis +Filter t1a#x IN (list#x []) +: +- Project [t2a#x] +: +- Filter t2b#x IN (list#x [t2c#x]) +: : +- Project [t3b#x] +: : +- Filter (outer(t2c#x) = t3c#x) +: : +- SubqueryAlias t3 +: : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) +: : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] +: : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: : +- SubqueryAlias t3 +: : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- SubqueryAlias t2 +: +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] ++- Aggregate [t1a#x], [t1a#x, max(t1b#x) AS max(t1b)#x] + +- Filter (cast(t1b#x as int) > 0) + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1c, + Min(t1d) +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + GROUP BY t2a + HAVING t2a > 'val2a') +GROUP BY t1a, t1c +HAVING Min(t1d) > t1c +-- !query analysis +Filter (min(t1d)#xL > cast(t1c#x as bigint)) ++- Aggregate [t1a#x, t1c#x], [t1a#x, t1c#x, min(t1d#xL) AS min(t1d)#xL] + +- Filter NOT t1a#x IN (list#x []) + : +- Filter (t2a#x > val2a) + : +- Aggregate [t2a#x], [t2a#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + WHERE t1a = t2a + GROUP BY t2c, t2d + HAVING t2c > 8) +GROUP BY t1a, t1b +HAVING t1b < 10 +-- !query analysis +Filter (cast(t1b#x as int) < 10) ++- Aggregate [t1a#x, t1b#x], [t1a#x, t1b#x] + +- Filter NOT t1d#xL IN (list#x [t1a#x]) + : +- Project [t2d#xL] + : +- Filter (t2c#x > 8) + : +- Aggregate [t2c#x, t2d#xL], [t2d#xL, t2c#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + Max(t1b) +FROM t1 +WHERE t1b > 0 +GROUP BY t1a +HAVING t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b > 3) +-- !query analysis +Filter NOT t1a#x IN (list#x []) +: +- Project [t2a#x] +: +- Filter (cast(t2b#x as int) > 3) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- SubqueryAlias t2 +: +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] ++- Aggregate [t1a#x], [t1a#x, max(t1b#x) AS max(t1b)#x] + +- Filter (cast(t1b#x as int) > 0) + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-joins.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-joins.sql.out new file mode 100644 index 0000000000000..b20e463652de6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-joins.sql.out @@ -0,0 +1,1201 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query analysis +CreateViewCommand `t1`, select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query analysis +CreateViewCommand `t2`, select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i), false, false, LocalTempView, true + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query analysis +CreateViewCommand `t3`, select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i), false, false, LocalTempView, true + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +create temporary view s1 as select * from values + (1), (3), (5), (7), (9) + as s1(id) +-- !query analysis +CreateViewCommand `s1`, select * from values + (1), (3), (5), (7), (9) + as s1(id), false, false, LocalTempView, true + +- Project [id#x] + +- SubqueryAlias s1 + +- LocalRelation [id#x] + + +-- !query +create temporary view s2 as select * from values + (1), (3), (4), (6), (9) + as s2(id) +-- !query analysis +CreateViewCommand `s2`, select * from values + (1), (3), (4), (6), (9) + as s2(id), false, false, LocalTempView, true + +- Project [id#x] + +- SubqueryAlias s2 + +- LocalRelation [id#x] + + +-- !query +create temporary view s3 as select * from values + (3), (4), (6), (9) + as s3(id) +-- !query analysis +CreateViewCommand `s3`, select * from values + (3), (4), (6), (9) + as s3(id), false, false, LocalTempView, true + +- Project [id#x] + +- SubqueryAlias s3 + +- LocalRelation [id#x] + + +-- !query +SELECT t1a, t1b, t1c, t3a, t3b, t3c +FROM t1 natural JOIN t3 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1a = t2a) + AND t1b = t3b + AND t1a = t3a +ORDER BY t1a, + t1b, + t1c DESC nulls first +-- !query analysis +Sort [t1a#x ASC NULLS FIRST, t1b#x ASC NULLS FIRST, t1c#x DESC NULLS FIRST], true ++- Project [t1a#x, t1b#x, t1c#x, t3a#x, t3b#x, t3c#x] + +- Filter ((t1a#x IN (list#x [t1a#x]) AND (t1b#x = t3b#x)) AND (t1a#x = t3a#x)) + : +- Project [t2a#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x, t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- Join Inner + :- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t3 + +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT Count(DISTINCT(t1a)), + t1b, + t3a, + t3b, + t3c +FROM t1 natural left JOIN t3 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1d = t2d) +AND t1b > t3b +GROUP BY t1a, + t1b, + t3a, + t3b, + t3c +ORDER BY t1a DESC, t3b DESC, t3c ASC +-- !query analysis +Project [count(DISTINCT t1a)#xL, t1b#x, t3a#x, t3b#x, t3c#x] ++- Sort [t1a#x DESC NULLS LAST, t3b#x DESC NULLS LAST, t3c#x ASC NULLS FIRST], true + +- Aggregate [t1a#x, t1b#x, t3a#x, t3b#x, t3c#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x, t3a#x, t3b#x, t3c#x, t1a#x] + +- Filter (t1a#x IN (list#x [t1d#xL]) AND (t1b#x > t3b#x)) + : +- Project [t2a#x] + : +- Filter (outer(t1d#xL) = t2d#xL) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x, t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- Join LeftOuter + :- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t3 + +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT Count(DISTINCT(t1a)) +FROM t1 natural right JOIN t3 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1b = t2b) +AND t1d IN + ( + SELECT t2d + FROM t2 + WHERE t1c > t2c) +AND t1a = t3a +GROUP BY t1a +ORDER BY t1a +-- !query analysis +Project [count(DISTINCT t1a)#xL] ++- Sort [t1a#x ASC NULLS FIRST], true + +- Aggregate [t1a#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1a#x] + +- Filter ((t1a#x IN (list#x [t1b#x]) AND t1d#xL IN (list#x [t1c#x])) AND (t1a#x = t3a#x)) + : :- Project [t2a#x] + : : +- Filter (outer(t1b#x) = t2b#x) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Project [t2d#xL] + : +- Filter (outer(t1c#x) > t2c#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x, t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- Join RightOuter + :- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t3 + +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT t1a, + t1b, + t1c, + t3a, + t3b, + t3c +FROM t1 FULL OUTER JOIN t3 +where t1a IN + ( + SELECT t2a + FROM t2 + WHERE t2c IS NOT NULL) +AND t1b != t3b +AND t1a = 'val1b' +ORDER BY t1a +-- !query analysis +Sort [t1a#x ASC NULLS FIRST], true ++- Project [t1a#x, t1b#x, t1c#x, t3a#x, t3b#x, t3c#x] + +- Filter ((t1a#x IN (list#x []) AND NOT (t1b#x = t3b#x)) AND (t1a#x = val1b)) + : +- Project [t2a#x] + : +- Filter isnotnull(t2c#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Join FullOuter + :- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t3 + +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 RIGHT JOIN t3 +where t1a IN + ( + SELECT t2a + FROM t2 + WHERE t2h > t3h) +AND t3a IN + ( + SELECT t2a + FROM t2 + WHERE t2c > t3c) +AND t1h >= t3h +GROUP BY t1a, + t1b +HAVING t1b > 8 +ORDER BY t1a +-- !query analysis +Project [count(DISTINCT t1a)#xL, t1b#x] ++- Sort [t1a#x ASC NULLS FIRST], true + +- Filter (cast(t1b#x as int) > 8) + +- Aggregate [t1a#x, t1b#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x, t1a#x] + +- Filter ((t1a#x IN (list#x [t3h#x]) AND t3a#x IN (list#x [t3c#x])) AND (t1h#x >= t3h#x)) + : :- Project [t2a#x] + : : +- Filter (t2h#x > outer(t3h#x)) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Project [t2a#x] + : +- Filter (t2c#x > outer(t3c#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Join RightOuter + :- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t3 + +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT Count(DISTINCT(t1a)) +FROM t1 LEFT OUTER +JOIN t3 +ON t1a = t3a +WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1h < t2h ) +GROUP BY t1a +ORDER BY t1a +-- !query analysis +Project [count(DISTINCT t1a)#xL] ++- Sort [t1a#x ASC NULLS FIRST], true + +- Aggregate [t1a#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1a#x] + +- Filter t1a#x IN (list#x [t1h#x]) + : +- Project [t2a#x] + : +- Filter (outer(t1h#x) < t2h#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Join LeftOuter, (t1a#x = t3a#x) + :- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t3 + +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 INNER JOIN t2 +ON t1a > t2a +WHERE t1b IN + ( + SELECT t2b + FROM t2 + WHERE t2h > t1h) +OR t1a IN + ( + SELECT t2a + FROM t2 + WHERE t2h < t1h) +GROUP BY t1b +HAVING t1b > 6 +-- !query analysis +Filter (cast(t1b#x as int) > 6) ++- Aggregate [t1b#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x] + +- Filter (t1b#x IN (list#x [t1h#x]) OR t1a#x IN (list#x [t1h#x])) + : :- Project [t2b#x] + : : +- Filter (t2h#x > outer(t1h#x)) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Project [t2a#x] + : +- Filter (t2h#x < outer(t1h#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Join Inner, (t1a#x > t2a#x) + :- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t2 + +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + JOIN t1 + WHERE t2b <> t1b) +AND t1h IN + ( + SELECT t2h + FROM t2 + RIGHT JOIN t3 + where t2b = t3b) +GROUP BY t1b +HAVING t1b > 8 +-- !query analysis +Filter (cast(t1b#x as int) > 8) ++- Aggregate [t1b#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x] + +- Filter (t1a#x IN (list#x []) AND t1h#x IN (list#x [])) + : :- Project [t2a#x] + : : +- Filter NOT (t2b#x = t1b#x) + : : +- Join Inner + : : :- SubqueryAlias t2 + : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- SubqueryAlias t2 + : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- Project [t2h#x] + : +- Filter (t2b#x = t3b#x) + : +- Join RightOuter + : :- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + JOIN t1 + WHERE t2b <> t1b) +AND t1h IN + ( + SELECT t2h + FROM t2 + RIGHT JOIN t3 + where t2b = t3b) +AND t1b IN + ( + SELECT t2b + FROM t2 + FULL OUTER JOIN t3 + where t2b = t3b) + +GROUP BY t1b +HAVING t1b > 8 +-- !query analysis +Filter (cast(t1b#x as int) > 8) ++- Aggregate [t1b#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x] + +- Filter ((t1a#x IN (list#x []) AND t1h#x IN (list#x [])) AND t1b#x IN (list#x [])) + : :- Project [t2a#x] + : : +- Filter NOT (t2b#x = t1b#x) + : : +- Join Inner + : : :- SubqueryAlias t2 + : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- SubqueryAlias t2 + : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : :- Project [t2h#x] + : : +- Filter (t2b#x = t3b#x) + : : +- Join RightOuter + : : :- SubqueryAlias t2 + : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- SubqueryAlias t2 + : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- Project [t2b#x] + : +- Filter (t2b#x = t3b#x) + : +- Join FullOuter + : :- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 +INNER JOIN t2 on t1b = t2b +RIGHT JOIN t3 ON t1a = t3a +where t1a IN + ( + SELECT t2a + FROM t2 + FULL OUTER JOIN t3 + WHERE t2b > t3b) +AND t1c IN + ( + SELECT t3c + FROM t3 + LEFT OUTER JOIN t2 + ON t3a = t2a ) +AND t1b IN + ( + SELECT t3b + FROM t3 LEFT OUTER + JOIN t1 + WHERE t3c = t1c) + +AND t1a = t2a +GROUP BY t1b +ORDER BY t1b DESC +-- !query analysis +Sort [t1b#x DESC NULLS LAST], true ++- Aggregate [t1b#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x] + +- Filter ((t1a#x IN (list#x []) AND t1c#x IN (list#x [])) AND (t1b#x IN (list#x []) AND (t1a#x = t2a#x))) + : :- Project [t2a#x] + : : +- Filter (t2b#x > t3b#x) + : : +- Join FullOuter + : : :- SubqueryAlias t2 + : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- SubqueryAlias t2 + : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : :- Project [t3c#x] + : : +- Join LeftOuter, (t3a#x = t2a#x) + : : :- SubqueryAlias t3 + : : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : : +- SubqueryAlias t3 + : : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Project [t3b#x] + : +- Filter (t3c#x = t1c#x) + : +- Join LeftOuter + : :- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Join RightOuter, (t1a#x = t3a#x) + :- Join Inner, (t1b#x = t2b#x) + : :- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t3 + +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT t1a, + t1b, + t1c, + count(distinct(t2a)), + t2b, + t2c +FROM t1 +FULL JOIN t2 on t1a = t2a +RIGHT JOIN t3 on t1a = t3a +where t1a IN + ( + SELECT t2a + FROM t2 INNER + JOIN t3 + ON t2b < t3b + WHERE t2c IN + ( + SELECT t1c + FROM t1 + WHERE t1a = t2a)) +and t1a = t2a +Group By t1a, t1b, t1c, t2a, t2b, t2c +HAVING t2c IS NOT NULL +ORDER By t2b DESC nulls last +-- !query analysis +Sort [t2b#x DESC NULLS LAST], true ++- Filter isnotnull(t2c#x) + +- Aggregate [t1a#x, t1b#x, t1c#x, t2a#x, t2b#x, t2c#x], [t1a#x, t1b#x, t1c#x, count(distinct t2a#x) AS count(DISTINCT t2a)#xL, t2b#x, t2c#x] + +- Filter (t1a#x IN (list#x []) AND (t1a#x = t2a#x)) + : +- Project [t2a#x] + : +- Filter t2c#x IN (list#x [t2a#x]) + : : +- Project [t1c#x] + : : +- Filter (t1a#x = outer(t2a#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- Join Inner, (t2b#x < t3b#x) + : :- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- Join RightOuter, (t1a#x = t3a#x) + :- Join FullOuter, (t1a#x = t2a#x) + : :- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t3 + +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id IN (SELECT 9) +-- !query analysis +Project [id#x] ++- Join Inner, ((id#x = id#x) AND id#x IN (list#x [])) + : +- Project [9 AS 9#x] + : +- OneRowRelation + :- SubqueryAlias s1 + : +- View (`s1`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s1 + : +- LocalRelation [id#x] + +- SubqueryAlias s2 + +- View (`s2`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias s2 + +- LocalRelation [id#x] + + +-- !query +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id NOT IN (SELECT 9) +-- !query analysis +Project [id#x] ++- Join Inner, ((id#x = id#x) AND NOT id#x IN (list#x [])) + : +- Project [9 AS 9#x] + : +- OneRowRelation + :- SubqueryAlias s1 + : +- View (`s1`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s1 + : +- LocalRelation [id#x] + +- SubqueryAlias s2 + +- View (`s2`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias s2 + +- LocalRelation [id#x] + + +-- !query +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query analysis +Project [id#x] ++- Join Inner, ((id#x = id#x) AND id#x IN (list#x [])) + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- View (`s3`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- LocalRelation [id#x] + :- SubqueryAlias s1 + : +- View (`s1`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s1 + : +- LocalRelation [id#x] + +- SubqueryAlias s2 + +- View (`s2`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias s2 + +- LocalRelation [id#x] + + +-- !query +SELECT s1.id AS id2 FROM s1 +LEFT SEMI JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query analysis +Project [id#x AS id2#x] ++- Join LeftSemi, ((id#x = id#x) AND id#x IN (list#x [])) + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- View (`s3`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- LocalRelation [id#x] + :- SubqueryAlias s1 + : +- View (`s1`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s1 + : +- LocalRelation [id#x] + +- SubqueryAlias s2 + +- View (`s2`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias s2 + +- LocalRelation [id#x] + + +-- !query +SELECT s1.id as id2 FROM s1 +LEFT ANTI JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query analysis +Project [id#x AS id2#x] ++- Join LeftAnti, ((id#x = id#x) AND id#x IN (list#x [])) + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- View (`s3`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- LocalRelation [id#x] + :- SubqueryAlias s1 + : +- View (`s1`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s1 + : +- LocalRelation [id#x] + +- SubqueryAlias s2 + +- View (`s2`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias s2 + +- LocalRelation [id#x] + + +-- !query +SELECT s1.id, s2.id as id2 FROM s1 +LEFT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query analysis +Project [id#x, id#x AS id2#x] ++- Join LeftOuter, ((id#x = id#x) AND id#x IN (list#x [])) + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- View (`s3`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- LocalRelation [id#x] + :- SubqueryAlias s1 + : +- View (`s1`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s1 + : +- LocalRelation [id#x] + +- SubqueryAlias s2 + +- View (`s2`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias s2 + +- LocalRelation [id#x] + + +-- !query +SELECT s1.id, s2.id as id2 FROM s1 +RIGHT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query analysis +Project [id#x, id#x AS id2#x] ++- Join RightOuter, ((id#x = id#x) AND id#x IN (list#x [])) + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- View (`s3`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- LocalRelation [id#x] + :- SubqueryAlias s1 + : +- View (`s1`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s1 + : +- LocalRelation [id#x] + +- SubqueryAlias s2 + +- View (`s2`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias s2 + +- LocalRelation [id#x] + + +-- !query +SELECT s1.id, s2.id AS id2 FROM s1 +FULL OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query analysis +Project [id#x, id#x AS id2#x] ++- Join FullOuter, ((id#x = id#x) AND id#x IN (list#x [])) + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- View (`s3`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- LocalRelation [id#x] + :- SubqueryAlias s1 + : +- View (`s1`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s1 + : +- LocalRelation [id#x] + +- SubqueryAlias s2 + +- View (`s2`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias s2 + +- LocalRelation [id#x] + + +-- !query +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query analysis +Project [id#x] ++- Join Inner, ((id#x = id#x) AND NOT id#x IN (list#x [])) + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- View (`s3`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- LocalRelation [id#x] + :- SubqueryAlias s1 + : +- View (`s1`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s1 + : +- LocalRelation [id#x] + +- SubqueryAlias s2 + +- View (`s2`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias s2 + +- LocalRelation [id#x] + + +-- !query +SELECT s1.id AS id2 FROM s1 +LEFT SEMI JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query analysis +Project [id#x AS id2#x] ++- Join LeftSemi, ((id#x = id#x) AND NOT id#x IN (list#x [])) + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- View (`s3`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- LocalRelation [id#x] + :- SubqueryAlias s1 + : +- View (`s1`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s1 + : +- LocalRelation [id#x] + +- SubqueryAlias s2 + +- View (`s2`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias s2 + +- LocalRelation [id#x] + + +-- !query +SELECT s1.id AS id2 FROM s1 +LEFT ANTI JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query analysis +Project [id#x AS id2#x] ++- Join LeftAnti, ((id#x = id#x) AND NOT id#x IN (list#x [])) + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- View (`s3`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- LocalRelation [id#x] + :- SubqueryAlias s1 + : +- View (`s1`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s1 + : +- LocalRelation [id#x] + +- SubqueryAlias s2 + +- View (`s2`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias s2 + +- LocalRelation [id#x] + + +-- !query +SELECT s1.id, s2.id AS id2 FROM s1 +LEFT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query analysis +Project [id#x, id#x AS id2#x] ++- Join LeftOuter, ((id#x = id#x) AND NOT id#x IN (list#x [])) + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- View (`s3`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- LocalRelation [id#x] + :- SubqueryAlias s1 + : +- View (`s1`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s1 + : +- LocalRelation [id#x] + +- SubqueryAlias s2 + +- View (`s2`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias s2 + +- LocalRelation [id#x] + + +-- !query +SELECT s1.id, s2.id AS id2 FROM s1 +RIGHT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query analysis +Project [id#x, id#x AS id2#x] ++- Join RightOuter, ((id#x = id#x) AND NOT id#x IN (list#x [])) + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- View (`s3`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- LocalRelation [id#x] + :- SubqueryAlias s1 + : +- View (`s1`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s1 + : +- LocalRelation [id#x] + +- SubqueryAlias s2 + +- View (`s2`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias s2 + +- LocalRelation [id#x] + + +-- !query +SELECT s1.id, s2.id AS id2 FROM s1 +FULL OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query analysis +Project [id#x, id#x AS id2#x] ++- Join FullOuter, ((id#x = id#x) AND NOT id#x IN (list#x [])) + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- View (`s3`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s3 + : +- LocalRelation [id#x] + :- SubqueryAlias s1 + : +- View (`s1`, [id#x]) + : +- Project [cast(id#x as int) AS id#x] + : +- Project [id#x] + : +- SubqueryAlias s1 + : +- LocalRelation [id#x] + +- SubqueryAlias s2 + +- View (`s2`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias s2 + +- LocalRelation [id#x] + + +-- !query +DROP VIEW s1 +-- !query analysis +DropTempViewCommand s1 + + +-- !query +DROP VIEW s2 +-- !query analysis +DropTempViewCommand s2 + + +-- !query +DROP VIEW s3 +-- !query analysis +DropTempViewCommand s3 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-limit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-limit.sql.out new file mode 100644 index 0000000000000..4308bad210930 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-limit.sql.out @@ -0,0 +1,589 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query analysis +CreateViewCommand `t1`, select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query analysis +CreateViewCommand `t2`, select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i), false, false, LocalTempView, true + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query analysis +CreateViewCommand `t3`, select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i), false, false, LocalTempView, true + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +LIMIT 2 +-- !query analysis +GlobalLimit 2 ++- LocalLimit 2 + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Filter t1a#x IN (list#x [t1d#xL]) + : +- Project [t2a#x] + : +- Filter (outer(t1d#xL) = t2d#xL) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b >= 8 + LIMIT 2) +LIMIT 4 +-- !query analysis +GlobalLimit 4 ++- LocalLimit 4 + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Filter t1c#x IN (list#x []) + : +- GlobalLimit 2 + : +- LocalLimit 2 + : +- Project [t2c#x] + : +- Filter (cast(t2b#x as int) >= 8) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + ORDER BY t2c, t2d + LIMIT 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +LIMIT 1 +-- !query analysis +GlobalLimit 1 ++- LocalLimit 1 + +- Sort [t1b#x DESC NULLS FIRST], true + +- Aggregate [t1b#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x] + +- Filter t1d#xL IN (list#x []) + : +- GlobalLimit 2 + : +- LocalLimit 2 + : +- Project [t2d#xL] + : +- Sort [t2c#x ASC NULLS FIRST, t2d#xL ASC NULLS FIRST], true + : +- Project [t2d#xL, t2c#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b > 6 + LIMIT 2) +-- !query analysis +Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Filter NOT t1b#x IN (list#x []) + : +- GlobalLimit 2 + : +- LocalLimit 2 + : +- Project [t2b#x] + : +- Filter (cast(t2b#x as int) > 6) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + ORDER BY t2b DESC nulls first, t2d + LIMIT 1) +GROUP BY t1b +ORDER BY t1b NULLS last +LIMIT 1 +-- !query analysis +GlobalLimit 1 ++- LocalLimit 1 + +- Sort [t1b#x ASC NULLS LAST], true + +- Aggregate [t1b#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x] + +- Filter NOT t1d#xL IN (list#x []) + : +- GlobalLimit 1 + : +- LocalLimit 1 + : +- Project [t2d#xL] + : +- Sort [t2b#x DESC NULLS FIRST, t2d#xL ASC NULLS FIRST], true + : +- Project [t2d#xL, t2b#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +LIMIT 2 +OFFSET 2 +-- !query analysis +GlobalLimit 2 ++- LocalLimit 2 + +- Offset 2 + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Filter t1a#x IN (list#x [t1d#xL]) + : +- Project [t2a#x] + : +- Filter (outer(t1d#xL) = t2d#xL) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b >= 8 + LIMIT 2 + OFFSET 2) +LIMIT 4 +OFFSET 2 +-- !query analysis +GlobalLimit 4 ++- LocalLimit 4 + +- Offset 2 + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Filter t1c#x IN (list#x []) + : +- GlobalLimit 2 + : +- LocalLimit 2 + : +- Offset 2 + : +- Project [t2c#x] + : +- Filter (cast(t2b#x as int) >= 8) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + ORDER BY t2c, t2d + LIMIT 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +LIMIT 1 +OFFSET 1 +-- !query analysis +GlobalLimit 1 ++- LocalLimit 1 + +- Offset 1 + +- Sort [t1b#x DESC NULLS FIRST], true + +- Aggregate [t1b#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x] + +- Filter t1d#xL IN (list#x []) + : +- GlobalLimit 2 + : +- LocalLimit 2 + : +- Project [t2d#xL] + : +- Sort [t2c#x ASC NULLS FIRST, t2d#xL ASC NULLS FIRST], true + : +- Project [t2d#xL, t2c#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b > 6 + LIMIT 2 + OFFSET 2) +-- !query analysis +Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Filter NOT t1b#x IN (list#x []) + : +- GlobalLimit 2 + : +- LocalLimit 2 + : +- Offset 2 + : +- Project [t2b#x] + : +- Filter (cast(t2b#x as int) > 6) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + ORDER BY t2b DESC nulls first, t2d + LIMIT 1 + OFFSET 1) +GROUP BY t1b +ORDER BY t1b NULLS last +LIMIT 1 +OFFSET 1 +-- !query analysis +GlobalLimit 1 ++- LocalLimit 1 + +- Offset 1 + +- Sort [t1b#x ASC NULLS LAST], true + +- Aggregate [t1b#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x] + +- Filter NOT t1d#xL IN (list#x []) + : +- GlobalLimit 1 + : +- LocalLimit 1 + : +- Offset 1 + : +- Project [t2d#xL] + : +- Sort [t2b#x DESC NULLS FIRST, t2d#xL ASC NULLS FIRST], true + : +- Project [t2d#xL, t2b#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +OFFSET 2 +-- !query analysis +Offset 2 ++- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Filter t1a#x IN (list#x [t1d#xL]) + : +- Project [t2a#x] + : +- Filter (outer(t1d#xL) = t2d#xL) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b >= 8 + OFFSET 2) +OFFSET 4 +-- !query analysis +Offset 4 ++- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Filter t1c#x IN (list#x []) + : +- Offset 2 + : +- Project [t2c#x] + : +- Filter (cast(t2b#x as int) >= 8) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + ORDER BY t2c, t2d + OFFSET 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +OFFSET 1 +-- !query analysis +Offset 1 ++- Sort [t1b#x DESC NULLS FIRST], true + +- Aggregate [t1b#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x] + +- Filter t1d#xL IN (list#x []) + : +- Offset 2 + : +- Project [t2d#xL] + : +- Sort [t2c#x ASC NULLS FIRST, t2d#xL ASC NULLS FIRST], true + : +- Project [t2d#xL, t2c#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b > 6 + OFFSET 2) +-- !query analysis +Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Filter NOT t1b#x IN (list#x []) + : +- Offset 2 + : +- Project [t2b#x] + : +- Filter (cast(t2b#x as int) > 6) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + ORDER BY t2b DESC nulls first, t2d + OFFSET 1) +GROUP BY t1b +ORDER BY t1b NULLS last +OFFSET 1 +-- !query analysis +Offset 1 ++- Sort [t1b#x ASC NULLS LAST], true + +- Aggregate [t1b#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x] + +- Filter NOT t1d#xL IN (list#x []) + : +- Offset 1 + : +- Project [t2d#xL] + : +- Sort [t2b#x DESC NULLS FIRST, t2d#xL ASC NULLS FIRST], true + : +- Project [t2d#xL, t2b#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out new file mode 100644 index 0000000000000..ab16f4b9d687c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out @@ -0,0 +1,336 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query analysis +CreateViewCommand `t1`, select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query analysis +CreateViewCommand `t2`, select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i), false, false, LocalTempView, true + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query analysis +CreateViewCommand `t3`, select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i), false, false, LocalTempView, true + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT t1a, + t1b, + t1h +FROM t1 +WHERE ( t1a, t1h ) NOT IN (SELECT t2a, + t2h + FROM t2 + WHERE t2a = t1a + ORDER BY t2a) +AND t1a = 'val1a' +-- !query analysis +Project [t1a#x, t1b#x, t1h#x] ++- Filter (NOT named_struct(t1a, t1a#x, t1h, t1h#x) IN (list#x [t1a#x]) AND (t1a#x = val1a)) + : +- Sort [t2a#x ASC NULLS FIRST], true + : +- Project [t2a#x, t2h#x] + : +- Filter (t2a#x = outer(t1a#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b, + t1d +FROM t1 +WHERE ( t1b, t1d ) IN (SELECT t2b, + t2d + FROM t2 + WHERE t2i IN (SELECT t3i + FROM t3 + WHERE t2b > t3b)) +-- !query analysis +Project [t1a#x, t1b#x, t1d#xL] ++- Filter named_struct(t1b, t1b#x, t1d, t1d#xL) IN (list#x []) + : +- Project [t2b#x, t2d#xL] + : +- Filter t2i#x IN (list#x [t2b#x]) + : : +- Project [t3i#x] + : : +- Filter (outer(t2b#x) > t3b#x) + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b, + t1d +FROM t1 +WHERE ( t1b, t1d ) NOT IN (SELECT t2b, + t2d + FROM t2 + WHERE t2h IN (SELECT t3h + FROM t3 + WHERE t2b > t3b)) +AND t1a = 'val1a' +-- !query analysis +Project [t1a#x, t1b#x, t1d#xL] ++- Filter (NOT named_struct(t1b, t1b#x, t1d, t1d#xL) IN (list#x []) AND (t1a#x = val1a)) + : +- Project [t2b#x, t2d#xL] + : +- Filter t2h#x IN (list#x [t2b#x]) + : : +- Project [t3h#x] + : : +- Filter (outer(t2b#x) > t3b#x) + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t2a +FROM (SELECT t2a + FROM t2 + WHERE ( t2a, t2b ) IN (SELECT t1a, + t1b + FROM t1) + UNION ALL + SELECT t2a + FROM t2 + WHERE ( t2a, t2b ) IN (SELECT t1a, + t1b + FROM t1) + UNION DISTINCT + SELECT t2a + FROM t2 + WHERE ( t2a, t2b ) IN (SELECT t3a, + t3b + FROM t3)) AS t4 +-- !query analysis +Project [t2a#x] ++- SubqueryAlias t4 + +- Distinct + +- Union false, false + :- Union false, false + : :- Project [t2a#x] + : : +- Filter named_struct(t2a, t2a#x, t2b, t2b#x) IN (list#x []) + : : : +- Project [t1a#x, t1b#x] + : : : +- SubqueryAlias t1 + : : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : : +- SubqueryAlias t1 + : : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Project [t2a#x] + : +- Filter named_struct(t2a, t2a#x, t2b, t2b#x) IN (list#x []) + : : +- Project [t1a#x, t1b#x] + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Project [t2a#x] + +- Filter named_struct(t2a, t2a#x, t2b, t2b#x) IN (list#x []) + : +- Project [t3a#x, t3b#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t2 + +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +WITH cte1 AS +( + SELECT t1a, + t1b + FROM t1 + WHERE ( + t1b, t1d) IN + ( + SELECT t2b, + t2d + FROM t2 + WHERE t1c = t2c)) +SELECT * +FROM ( + SELECT * + FROM cte1 + JOIN cte1 cte2 + on cte1.t1b = cte2.t1b) s +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte1 +: +- Project [t1a#x, t1b#x] +: +- Filter named_struct(t1b, t1b#x, t1d, t1d#xL) IN (list#x [t1c#x]) +: : +- Project [t2b#x, t2d#xL] +: : +- Filter (outer(t1c#x) = t2c#x) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Project [t1a#x, t1b#x, t1a#x, t1b#x] + +- SubqueryAlias s + +- Project [t1a#x, t1b#x, t1a#x, t1b#x] + +- Join Inner, (t1b#x = t1b#x) + :- SubqueryAlias cte1 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + +- SubqueryAlias cte2 + +- SubqueryAlias cte1 + +- CTERelationRef xxxx, true, [t1a#x, t1b#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-order-by.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-order-by.sql.out new file mode 100644 index 0000000000000..f424b8ae7cf84 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-order-by.sql.out @@ -0,0 +1,558 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query analysis +CreateViewCommand `t1`, select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query analysis +CreateViewCommand `t2`, select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i), false, false, LocalTempView, true + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query analysis +CreateViewCommand `t3`, select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i), false, false, LocalTempView, true + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2) +ORDER BY t1a +-- !query analysis +Sort [t1a#x ASC NULLS FIRST], true ++- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Filter t1a#x IN (list#x []) + : +- Project [t2a#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +ORDER BY t1b DESC +-- !query analysis +Project [t1a#x] ++- Sort [t1b#x DESC NULLS LAST], true + +- Project [t1a#x, t1b#x] + +- Filter t1b#x IN (list#x [t1a#x]) + : +- Project [t2b#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) +ORDER BY 2 DESC nulls last +-- !query analysis +Sort [t1b#x DESC NULLS LAST], true ++- Project [t1a#x, t1b#x] + +- Filter t1c#x IN (list#x [t1a#x]) + : +- Project [t2c#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT Count(DISTINCT( t1a )) +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +ORDER BY Count(DISTINCT( t1a )) +-- !query analysis +Sort [count(DISTINCT t1a)#xL ASC NULLS FIRST], true ++- Aggregate [count(distinct t1a#x) AS count(DISTINCT t1a)#xL] + +- Filter t1b#x IN (list#x [t1a#x]) + : +- Project [t2b#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1b IN (SELECT t2c + FROM t2 + ORDER BY t2d) +-- !query analysis +Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Filter cast(t1b#x as int) IN (list#x []) + : +- Project [t2c#x] + : +- Project [t2c#x] + : +- Sort [t2d#xL ASC NULLS FIRST], true + : +- Project [t2c#x, t2d#xL] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1b = t2b + ORDER BY Min(t2b)) +ORDER BY t1c DESC nulls first, t1a DESC, t1d DESC, t1h +-- !query analysis +Sort [t1c#x DESC NULLS FIRST, t1a#x DESC NULLS LAST, t1d#xL DESC NULLS LAST, t1h#x ASC NULLS FIRST], true ++- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Filter t1b#x IN (list#x [t1b#x]) + : +- Sort [min(t2b)#x ASC NULLS FIRST], true + : +- Aggregate [min(t2b#x) AS min(t2b)#x] + : +- Filter (outer(t1b#x) = t2b#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b, + t1h +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a + ORDER BY t2b DESC nulls first) + OR t1h IN (SELECT t2h + FROM t2 + WHERE t1h > t2h) +ORDER BY t1h DESC nulls last +-- !query analysis +Sort [t1h#x DESC NULLS LAST], true ++- Project [t1a#x, t1b#x, t1h#x] + +- Filter (t1c#x IN (list#x [t1a#x]) OR t1h#x IN (list#x [t1h#x])) + : :- Project [t2c#x] + : : +- Sort [t2b#x DESC NULLS FIRST], true + : : +- Project [t2c#x, t2b#x] + : : +- Filter (outer(t1a#x) = t2a#x) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Project [t2h#x] + : +- Filter (outer(t1h#x) > t2h#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2) +ORDER BY t1a +-- !query analysis +Sort [t1a#x ASC NULLS FIRST], true ++- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Filter NOT t1a#x IN (list#x []) + : +- Project [t2a#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t1a = t2a) +ORDER BY t1b DESC nulls last +-- !query analysis +Sort [t1b#x DESC NULLS LAST], true ++- Project [t1a#x, t1b#x] + +- Filter NOT t1a#x IN (list#x [t1a#x]) + : +- Project [t2a#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + ORDER BY t2a DESC nulls first) + and t1c IN (SELECT t2c + FROM t2 + ORDER BY t2b DESC nulls last) +ORDER BY t1c DESC nulls last +-- !query analysis +Sort [t1c#x DESC NULLS LAST], true ++- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Filter (NOT t1a#x IN (list#x []) AND t1c#x IN (list#x [])) + : :- Sort [t2a#x DESC NULLS FIRST], true + : : +- Project [t2a#x] + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Project [t2c#x] + : +- Sort [t2b#x DESC NULLS LAST], true + : +- Project [t2c#x, t2b#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + GROUP BY t2a + ORDER BY t2a DESC) +-- !query analysis +Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Filter t1b#x IN (list#x []) + : +- Project [min(t2b)#x] + : +- Sort [t2a#x DESC NULLS LAST], true + : +- Aggregate [t2a#x], [min(t2b#x) AS min(t2b)#x, t2a#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + Count(DISTINCT( t1b )) +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2a + ORDER BY t2a) +GROUP BY t1a, + t1h +ORDER BY t1a +-- !query analysis +Sort [t1a#x ASC NULLS FIRST], true ++- Aggregate [t1a#x, t1h#x], [t1a#x, count(distinct t1b#x) AS count(DISTINCT t1b)#xL] + +- Filter t1b#x IN (list#x [t1a#x]) + : +- Project [min(t2b)#x] + : +- Sort [t2a#x ASC NULLS FIRST], true + : +- Aggregate [t2a#x], [min(t2b#x) AS min(t2b)#x, t2a#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT Min(t2b) + FROM t2 + GROUP BY t2a + ORDER BY t2a) +-- !query analysis +Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Filter NOT t1b#x IN (list#x []) + : +- Project [min(t2b)#x] + : +- Sort [t2a#x ASC NULLS FIRST], true + : +- Aggregate [t2a#x], [min(t2b#x) AS min(t2b)#x, t2a#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1b NOT IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2c + ORDER BY t2c DESC nulls last) +GROUP BY t1a +-- !query analysis +Aggregate [t1a#x], [t1a#x, sum(distinct t1b#x) AS sum(DISTINCT t1b)#xL] ++- Filter NOT t1b#x IN (list#x [t1a#x]) + : +- Project [min(t2b)#x] + : +- Sort [t2c#x DESC NULLS LAST], true + : +- Aggregate [t2c#x], [min(t2b#x) AS min(t2b)#x, t2c#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1h NOT IN (SELECT t2h + FROM t2 + where t1a = t2a + order by t2d DESC nulls first + ) +GROUP BY t1a, + t1b +ORDER BY t1b DESC nulls last +-- !query analysis +Sort [t1b#x DESC NULLS LAST], true ++- Aggregate [t1a#x, t1b#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x] + +- Filter NOT t1h#x IN (list#x [t1a#x]) + : +- Project [t2h#x] + : +- Sort [t2d#xL DESC NULLS FIRST], true + : +- Project [t2h#x, t2d#xL] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-set-operations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-set-operations.sql.out new file mode 100644 index 0000000000000..937173cac7438 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-set-operations.sql.out @@ -0,0 +1,1627 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query analysis +CreateViewCommand `t1`, select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query analysis +CreateViewCommand `t2`, select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i), false, false, LocalTempView, true + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query analysis +CreateViewCommand `t3`, select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i), false, false, LocalTempView, true + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT t2a, + t2b, + t2c, + t2h, + t2i +FROM (SELECT * + FROM t2 + WHERE t2a IN (SELECT t1a + FROM t1) + UNION ALL + SELECT * + FROM t3 + WHERE t3a IN (SELECT t1a + FROM t1)) AS t3 +WHERE t2i IS NOT NULL AND + 2 * t2b = t2c +ORDER BY t2c DESC nulls first +-- !query analysis +Sort [t2c#x DESC NULLS FIRST], true ++- Project [t2a#x, t2b#x, t2c#x, t2h#x, t2i#x] + +- Filter (isnotnull(t2i#x) AND ((2 * cast(t2b#x as int)) = t2c#x)) + +- SubqueryAlias t3 + +- Union false, false + :- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Filter t2a#x IN (list#x []) + : : +- Project [t1a#x] + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- Filter t3a#x IN (list#x []) + : +- Project [t1a#x] + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t3 + +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT t2a, + t2b, + t2d, + Count(DISTINCT( t2h )), + t2i +FROM (SELECT * + FROM t2 + WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t2b = t1b) + UNION + SELECT * + FROM t1 + WHERE t1a IN (SELECT t3a + FROM t3 + WHERE t1c = t3c)) AS t3 +GROUP BY t2a, + t2b, + t2d, + t2i +ORDER BY t2d DESC +-- !query analysis +Sort [t2d#xL DESC NULLS LAST], true ++- Aggregate [t2a#x, t2b#x, t2d#xL, t2i#x], [t2a#x, t2b#x, t2d#xL, count(distinct t2h#x) AS count(DISTINCT t2h)#xL, t2i#x] + +- SubqueryAlias t3 + +- Distinct + +- Union false, false + :- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Filter t2a#x IN (list#x [t2b#x]) + : : +- Project [t1a#x] + : : +- Filter (outer(t2b#x) = t1b#x) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Filter t1a#x IN (list#x [t1c#x]) + : +- Project [t3a#x] + : +- Filter (outer(t1c#x) = t3c#x) + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t2a, + t2b, + t2c, + Min(t2d) +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b) +GROUP BY t2a, t2b, t2c +UNION ALL +SELECT t2a, + t2b, + t2c, + Max(t2d) +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t2c = t1c) +GROUP BY t2a, t2b, t2c +UNION +SELECT t3a, + t3b, + t3c, + Min(t3d) +FROM t3 +WHERE t3a IN (SELECT t2a + FROM t2 + WHERE t3c = t2c) +GROUP BY t3a, t3b, t3c +UNION DISTINCT +SELECT t1a, + t1b, + t1c, + Max(t1d) +FROM t1 +WHERE t1a IN (SELECT t3a + FROM t3 + WHERE t3d = t1d) +GROUP BY t1a, t1b, t1c +-- !query analysis +Distinct ++- Union false, false + :- Distinct + : +- Union false, false + : :- Union false, false + : : :- Aggregate [t2a#x, t2b#x, t2c#x], [t2a#x, t2b#x, t2c#x, min(t2d#xL) AS min(t2d)#xL] + : : : +- Filter t2a#x IN (list#x [t2b#x]) + : : : : +- Project [t1a#x] + : : : : +- Filter (t1b#x = outer(t2b#x)) + : : : : +- SubqueryAlias t1 + : : : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : : : +- SubqueryAlias t1 + : : : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : : +- SubqueryAlias t2 + : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- SubqueryAlias t2 + : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- Aggregate [t2a#x, t2b#x, t2c#x], [t2a#x, t2b#x, t2c#x, max(t2d#xL) AS max(t2d)#xL] + : : +- Filter t2a#x IN (list#x [t2c#x]) + : : : +- Project [t1a#x] + : : : +- Filter (outer(t2c#x) = t1c#x) + : : : +- SubqueryAlias t1 + : : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : : +- SubqueryAlias t1 + : : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Aggregate [t3a#x, t3b#x, t3c#x], [t3a#x, t3b#x, t3c#x, min(t3d#xL) AS min(t3d)#xL] + : +- Filter t3a#x IN (list#x [t3c#x]) + : : +- Project [t2a#x] + : : +- Filter (outer(t3c#x) = t2c#x) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- Aggregate [t1a#x, t1b#x, t1c#x], [t1a#x, t1b#x, t1c#x, max(t1d#xL) AS max(t1d)#xL] + +- Filter t1a#x IN (list#x [t1d#xL]) + : +- Project [t3a#x] + : +- Filter (t3d#xL = outer(t1d#xL)) + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT DISTINCT( t2a ), + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i +UNION +SELECT DISTINCT( t2a ), + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t2c = t1c) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i +HAVING t2b IS NOT NULL +-- !query analysis +Distinct ++- Union false, false + :- Distinct + : +- Aggregate [t2a#x, t2b#x, t2c#x, t2d#xL, t2h#x, t2i#x], [t2a#x, t2b#x, count(t2c#x) AS count(t2c)#xL, t2d#xL, t2h#x, t2i#x] + : +- Filter t2a#x IN (list#x [t2b#x]) + : : +- Project [t1a#x] + : : +- Filter (t1b#x = outer(t2b#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Distinct + +- Filter isnotnull(t2b#x) + +- Aggregate [t2a#x, t2b#x, t2c#x, t2d#xL, t2h#x, t2i#x], [t2a#x, t2b#x, count(t2c#x) AS count(t2c)#xL, t2d#xL, t2h#x, t2i#x] + +- Filter t2a#x IN (list#x [t2c#x]) + : +- Project [t1a#x] + : +- Filter (outer(t2c#x) = t1c#x) + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t2 + +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +SELECT t2a, + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2a IN (SELECT DISTINCT(t1a) + FROM t1 + WHERE t1b = t2b) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i + +UNION +SELECT DISTINCT( t2a ), + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2b IN (SELECT Max(t1b) + FROM t1 + WHERE t2c = t1c) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i +HAVING t2b IS NOT NULL +UNION DISTINCT +SELECT t2a, + t2b, + t2c, + t2d, + t2h, + t2i +FROM t2 +WHERE t2d IN (SELECT min(t1d) + FROM t1 + WHERE t2c = t1c) +-- !query analysis +Distinct ++- Union false, false + :- Distinct + : +- Union false, false + : :- Aggregate [t2a#x, t2b#x, t2c#x, t2d#xL, t2h#x, t2i#x], [t2a#x, t2b#x, count(t2c#x) AS count(t2c)#xL, t2d#xL, t2h#x, t2i#x] + : : +- Filter t2a#x IN (list#x [t2b#x]) + : : : +- Distinct + : : : +- Project [t1a#x] + : : : +- Filter (t1b#x = outer(t2b#x)) + : : : +- SubqueryAlias t1 + : : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : : +- SubqueryAlias t1 + : : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Distinct + : +- Filter isnotnull(t2b#x) + : +- Aggregate [t2a#x, t2b#x, t2c#x, t2d#xL, t2h#x, t2i#x], [t2a#x, t2b#x, count(t2c#x) AS count(t2c)#xL, t2d#xL, t2h#x, t2i#x] + : +- Filter t2b#x IN (list#x [t2c#x]) + : : +- Aggregate [max(t1b#x) AS max(t1b)#x] + : : +- Filter (outer(t2c#x) = t1c#x) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Project [t2a#x, t2b#x, cast(t2c#x as bigint) AS t2c#xL, t2d#xL, t2h#x, t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2h#x, t2i#x] + +- Filter t2d#xL IN (list#x [t2c#x]) + : +- Aggregate [min(t1d#xL) AS min(t1d)#xL] + : +- Filter (outer(t2c#x) = t1c#x) + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t2 + +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b AND + t1d < t2d) +INTERSECT +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2b IN (SELECT Max(t1b) + FROM t1 + WHERE t2c = t1c) +EXCEPT +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2d IN (SELECT Min(t3d) + FROM t3 + WHERE t2c = t3c) +UNION ALL +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2c IN (SELECT Max(t1c) + FROM t1 + WHERE t1d = t2d) +-- !query analysis +Union false, false +:- Except false +: :- Intersect false +: : :- Project [t2a#x, t2b#x, t2c#x, t2d#xL] +: : : +- Filter t2a#x IN (list#x [t2b#x && t2d#xL]) +: : : : +- Project [t1a#x] +: : : : +- Filter ((t1b#x = outer(t2b#x)) AND (t1d#xL < outer(t2d#xL))) +: : : : +- SubqueryAlias t1 +: : : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: : : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: : : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: : : : +- SubqueryAlias t1 +: : : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: : : +- SubqueryAlias t2 +: : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : : +- SubqueryAlias t2 +: : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL] +: : +- Filter t2b#x IN (list#x [t2c#x]) +: : : +- Aggregate [max(t1b#x) AS max(t1b)#x] +: : : +- Filter (outer(t2c#x) = t1c#x) +: : : +- SubqueryAlias t1 +: : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: : : +- SubqueryAlias t1 +: : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: : +- SubqueryAlias t2 +: : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- Project [t2a#x, t2b#x, t2c#x, t2d#xL] +: +- Filter t2d#xL IN (list#x [t2c#x]) +: : +- Aggregate [min(t3d#xL) AS min(t3d)#xL] +: : +- Filter (outer(t2c#x) = t3c#x) +: : +- SubqueryAlias t3 +: : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) +: : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] +: : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: : +- SubqueryAlias t3 +: : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- SubqueryAlias t2 +: +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] ++- Project [t2a#x, t2b#x, t2c#x, t2d#xL] + +- Filter t2c#x IN (list#x [t2d#xL]) + : +- Aggregate [max(t1c#x) AS max(t1c)#x] + : +- Filter (t1d#xL = outer(t2d#xL)) + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t2 + +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +SELECT DISTINCT(t1a), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a IN (SELECT t3a + FROM (SELECT t2a t3a + FROM t2 + UNION ALL + SELECT t2a t3a + FROM t2) AS t3 + UNION + SELECT t2a + FROM (SELECT t2a + FROM t2 + WHERE t2b > 6 + UNION + SELECT t2a + FROM t2 + WHERE t2b > 6) AS t4 + UNION DISTINCT + SELECT t2a + FROM (SELECT t2a + FROM t2 + WHERE t2b > 6 + UNION DISTINCT + SELECT t1a + FROM t1 + WHERE t1b > 6) AS t5) +GROUP BY t1a, t1b, t1c, t1d +HAVING t1c IS NOT NULL AND t1b IS NOT NULL +ORDER BY t1c DESC, t1a DESC +-- !query analysis +Sort [t1c#x DESC NULLS LAST, t1a#x DESC NULLS LAST], true ++- Distinct + +- Filter (isnotnull(t1c#x) AND isnotnull(t1b#x)) + +- Aggregate [t1a#x, t1b#x, t1c#x, t1d#xL], [t1a#x, t1b#x, t1c#x, t1d#xL] + +- Filter t1a#x IN (list#x []) + : +- Distinct + : +- Union false, false + : :- Distinct + : : +- Union false, false + : : :- Project [t3a#x] + : : : +- SubqueryAlias t3 + : : : +- Union false, false + : : : :- Project [t2a#x AS t3a#x] + : : : : +- SubqueryAlias t2 + : : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : : +- SubqueryAlias t2 + : : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- Project [t2a#x AS t3a#x] + : : : +- SubqueryAlias t2 + : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- SubqueryAlias t2 + : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- Project [t2a#x] + : : +- SubqueryAlias t4 + : : +- Distinct + : : +- Union false, false + : : :- Project [t2a#x] + : : : +- Filter (cast(t2b#x as int) > 6) + : : : +- SubqueryAlias t2 + : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- SubqueryAlias t2 + : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- Project [t2a#x] + : : +- Filter (cast(t2b#x as int) > 6) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Project [t2a#x] + : +- SubqueryAlias t5 + : +- Distinct + : +- Union false, false + : :- Project [t2a#x] + : : +- Filter (cast(t2b#x as int) > 6) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Project [t1a#x] + : +- Filter (cast(t1b#x as int) > 6) + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN (SELECT t2b + FROM (SELECT t2b + FROM t2 + WHERE t2b > 6 + INTERSECT + SELECT t1b + FROM t1 + WHERE t1b > 6) AS t3 + WHERE t2b = t1b) +-- !query analysis +Project [t1a#x, t1b#x, t1c#x] ++- Filter t1b#x IN (list#x [t1b#x]) + : +- Project [t2b#x] + : +- Filter (t2b#x = outer(t1b#x)) + : +- SubqueryAlias t3 + : +- Intersect false + : :- Project [t2b#x] + : : +- Filter (cast(t2b#x as int) > 6) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Project [t1b#x] + : +- Filter (cast(t1b#x as int) > 6) + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1h IN (SELECT t2h + FROM (SELECT t2h + FROM t2 + EXCEPT + SELECT t3h + FROM t3) AS t3) +ORDER BY t1b DESC NULLs first, t1c DESC NULLs last +-- !query analysis +Sort [t1b#x DESC NULLS FIRST, t1c#x DESC NULLS LAST], true ++- Project [t1a#x, t1b#x, t1c#x] + +- Filter t1h#x IN (list#x []) + : +- Project [t2h#x] + : +- SubqueryAlias t3 + : +- Except false + : :- Project [t2h#x] + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Project [t3h#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN + ( + SELECT t2b + FROM ( + SELECT t2b + FROM t2 + WHERE t2b > 6 + INTERSECT + SELECT t1b + FROM t1 + WHERE t1b > 6) AS t3) +UNION DISTINCT +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN + ( + SELECT t2b + FROM ( + SELECT t2b + FROM t2 + WHERE t2b > 6 + EXCEPT + SELECT t1b + FROM t1 + WHERE t1b > 6) AS t4 + WHERE t2b = t1b) +ORDER BY t1c DESC NULLS last, t1a DESC +-- !query analysis +Sort [t1c#x DESC NULLS LAST, t1a#x DESC NULLS LAST], true ++- Distinct + +- Union false, false + :- Project [t1a#x, t1b#x, t1c#x] + : +- Filter t1b#x IN (list#x []) + : : +- Project [t2b#x] + : : +- SubqueryAlias t3 + : : +- Intersect false + : : :- Project [t2b#x] + : : : +- Filter (cast(t2b#x as int) > 6) + : : : +- SubqueryAlias t2 + : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- SubqueryAlias t2 + : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- Project [t1b#x] + : : +- Filter (cast(t1b#x as int) > 6) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Project [t1a#x, t1b#x, t1c#x] + +- Filter t1b#x IN (list#x [t1b#x]) + : +- Project [t2b#x] + : +- Filter (t2b#x = outer(t1b#x)) + : +- SubqueryAlias t4 + : +- Except false + : :- Project [t2b#x] + : : +- Filter (cast(t2b#x as int) > 6) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Project [t1b#x] + : +- Filter (cast(t1b#x as int) > 6) + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM (SELECT * + FROM (SELECT * + FROM t2 + WHERE t2h IN (SELECT t1h + FROM t1 + WHERE t1a = t2a) + UNION DISTINCT + SELECT * + FROM t1 + WHERE t1h IN (SELECT t3h + FROM t3 + UNION + SELECT t1h + FROM t1) + UNION + SELECT * + FROM t3 + WHERE t3a IN (SELECT t2a + FROM t2 + UNION ALL + SELECT t1a + FROM t1 + WHERE t1b > 0) + INTERSECT + SELECT * + FROM T1 + WHERE t1b IN (SELECT t3b + FROM t3 + UNION DISTINCT + SELECT t2b + FROM t2 + ) + EXCEPT + SELECT * + FROM t2 + WHERE t2h IN (SELECT t1i + FROM t1)) t4 + WHERE t4.t2b IN (SELECT Min(t3b) + FROM t3 + WHERE t4.t2a = t3a)) +-- !query analysis +Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Filter t2b#x IN (list#x [t2a#x]) + : +- Aggregate [min(t3b#x) AS min(t3b)#x] + : +- Filter (outer(t2a#x) = t3a#x) + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t4 + +- Except false + :- Distinct + : +- Union false, false + : :- Distinct + : : +- Union false, false + : : :- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- Filter t2h#x IN (list#x [t2a#x]) + : : : : +- Project [t1h#x] + : : : : +- Filter (t1a#x = outer(t2a#x)) + : : : : +- SubqueryAlias t1 + : : : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : : : +- SubqueryAlias t1 + : : : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : : +- SubqueryAlias t2 + : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- SubqueryAlias t2 + : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- Filter t1h#x IN (list#x []) + : : : +- Distinct + : : : +- Union false, false + : : : :- Project [t3h#x] + : : : : +- SubqueryAlias t3 + : : : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : : : +- SubqueryAlias t3 + : : : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : : +- Project [t1h#x] + : : : +- SubqueryAlias t1 + : : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : : +- SubqueryAlias t1 + : : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- Intersect false + : :- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- Filter t3a#x IN (list#x []) + : : : +- Union false, false + : : : :- Project [t2a#x] + : : : : +- SubqueryAlias t2 + : : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : : +- SubqueryAlias t2 + : : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- Project [t1a#x] + : : : +- Filter (cast(t1b#x as int) > 0) + : : : +- SubqueryAlias t1 + : : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : : +- SubqueryAlias t1 + : : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- Filter t1b#x IN (list#x []) + : : +- Distinct + : : +- Union false, false + : : :- Project [t3b#x] + : : : +- SubqueryAlias t3 + : : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : : +- SubqueryAlias t3 + : : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- Project [t2b#x] + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Filter t2h#x IN (list#x []) + : +- Project [cast(t1i#x as timestamp) AS t1i#x] + : +- Project [t1i#x] + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t2 + +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +SELECT t2a, + t2b, + t2c, + t2i +FROM (SELECT * + FROM t2 + WHERE t2a NOT IN (SELECT t1a + FROM t1 + UNION + SELECT t3a + FROM t3) + UNION ALL + SELECT * + FROM t2 + WHERE t2a NOT IN (SELECT t1a + FROM t1 + INTERSECT + SELECT t2a + FROM t2)) AS t3 +WHERE t3.t2a NOT IN (SELECT t1a + FROM t1 + INTERSECT + SELECT t2a + FROM t2) + AND t2c IS NOT NULL +ORDER BY t2a +-- !query analysis +Sort [t2a#x ASC NULLS FIRST], true ++- Project [t2a#x, t2b#x, t2c#x, t2i#x] + +- Filter (NOT t2a#x IN (list#x []) AND isnotnull(t2c#x)) + : +- Intersect false + : :- Project [t1a#x] + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- Project [t2a#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t3 + +- Union false, false + :- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Filter NOT t2a#x IN (list#x []) + : : +- Distinct + : : +- Union false, false + : : :- Project [t1a#x] + : : : +- SubqueryAlias t1 + : : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : : +- SubqueryAlias t1 + : : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- Project [t3a#x] + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Filter NOT t2a#x IN (list#x []) + : +- Intersect false + : :- Project [t1a#x] + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- Project [t2a#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +SELECT Count(DISTINCT(t1a)), + t1b, + t1c, + t1i +FROM t1 +WHERE t1b NOT IN + ( + SELECT t2b + FROM ( + SELECT t2b + FROM t2 + WHERE t2b NOT IN + ( + SELECT t1b + FROM t1) + UNION + SELECT t1b + FROM t1 + WHERE t1b NOT IN + ( + SELECT t3b + FROM t3) + UNION + distinct SELECT t3b + FROM t3 + WHERE t3b NOT IN + ( + SELECT t2b + FROM t2)) AS t3 + WHERE t2b = t1b) +GROUP BY t1a, + t1b, + t1c, + t1i +HAVING t1b NOT IN + ( + SELECT t2b + FROM t2 + WHERE t2c IS NULL + EXCEPT + SELECT t3b + FROM t3) +ORDER BY t1c DESC NULLS LAST, t1i +-- !query analysis +Sort [t1c#x DESC NULLS LAST, t1i#x ASC NULLS FIRST], true ++- Filter NOT t1b#x IN (list#x []) + : +- Except false + : :- Project [t2b#x] + : : +- Filter isnull(t2c#x) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Project [t3b#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- Aggregate [t1a#x, t1b#x, t1c#x, t1i#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x, t1c#x, t1i#x] + +- Filter NOT t1b#x IN (list#x [t1b#x]) + : +- Project [t2b#x] + : +- Filter (t2b#x = outer(t1b#x)) + : +- SubqueryAlias t3 + : +- Distinct + : +- Union false, false + : :- Distinct + : : +- Union false, false + : : :- Project [t2b#x] + : : : +- Filter NOT t2b#x IN (list#x []) + : : : : +- Project [t1b#x] + : : : : +- SubqueryAlias t1 + : : : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : : : +- SubqueryAlias t1 + : : : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : : +- SubqueryAlias t2 + : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- SubqueryAlias t2 + : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- Project [t1b#x] + : : +- Filter NOT t1b#x IN (list#x []) + : : : +- Project [t3b#x] + : : : +- SubqueryAlias t3 + : : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : : +- SubqueryAlias t3 + : : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- Project [t3b#x] + : +- Filter NOT t3b#x IN (list#x []) + : : +- Project [t2b#x] + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + UNION ALL + SELECT t3a + FROM t3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (t2b#x = outer(t1b#x))\n+- SubqueryAlias t2\n +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x])\n +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x]\n +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n +- SubqueryAlias t2\n +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 35, + "stopIndex" : 100, + "fragment" : "SELECT t2a\n FROM t2\n WHERE t2b = t1b" + } ] +} + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + UNION DISTINCT + SELECT t3a + FROM t3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (t2b#x = outer(t1b#x))\n+- SubqueryAlias t2\n +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x])\n +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x]\n +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n +- SubqueryAlias t2\n +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 35, + "stopIndex" : 100, + "fragment" : "SELECT t2a\n FROM t2\n WHERE t2b = t1b" + } ] +} + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + INTERSECT ALL + SELECT t3a + FROM t3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (t2b#x = outer(t1b#x))\n+- SubqueryAlias t2\n +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x])\n +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x]\n +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n +- SubqueryAlias t2\n +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 35, + "stopIndex" : 100, + "fragment" : "SELECT t2a\n FROM t2\n WHERE t2b = t1b" + } ] +} + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + INTERSECT DISTINCT + SELECT t3a + FROM t3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (t2b#x = outer(t1b#x))\n+- SubqueryAlias t2\n +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x])\n +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x]\n +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n +- SubqueryAlias t2\n +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 35, + "stopIndex" : 100, + "fragment" : "SELECT t2a\n FROM t2\n WHERE t2b = t1b" + } ] +} + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + EXCEPT ALL + SELECT t3a + FROM t3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (t2b#x = outer(t1b#x))\n+- SubqueryAlias t2\n +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x])\n +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x]\n +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n +- SubqueryAlias t2\n +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 35, + "stopIndex" : 100, + "fragment" : "SELECT t2a\n FROM t2\n WHERE t2b = t1b" + } ] +} + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + EXCEPT DISTINCT + SELECT t3a + FROM t3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (t2b#x = outer(t1b#x))\n+- SubqueryAlias t2\n +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x])\n +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x]\n +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n +- SubqueryAlias t2\n +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 35, + "stopIndex" : 100, + "fragment" : "SELECT t2a\n FROM t2\n WHERE t2b = t1b" + } ] +} + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + UNION ALL + SELECT t3a + FROM t3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (t2b#x = outer(t1b#x))\n+- SubqueryAlias t2\n +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x])\n +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x]\n +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n +- SubqueryAlias t2\n +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 104, + "fragment" : "SELECT t2a\n FROM t2\n WHERE t2b = t1b" + } ] +} + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + UNION DISTINCT + SELECT t3a + FROM t3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (t2b#x = outer(t1b#x))\n+- SubqueryAlias t2\n +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x])\n +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x]\n +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n +- SubqueryAlias t2\n +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 104, + "fragment" : "SELECT t2a\n FROM t2\n WHERE t2b = t1b" + } ] +} + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + INTERSECT ALL + SELECT t3a + FROM t3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (t2b#x = outer(t1b#x))\n+- SubqueryAlias t2\n +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x])\n +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x]\n +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n +- SubqueryAlias t2\n +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 104, + "fragment" : "SELECT t2a\n FROM t2\n WHERE t2b = t1b" + } ] +} + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + INTERSECT DISTINCT + SELECT t3a + FROM t3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (t2b#x = outer(t1b#x))\n+- SubqueryAlias t2\n +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x])\n +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x]\n +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n +- SubqueryAlias t2\n +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 104, + "fragment" : "SELECT t2a\n FROM t2\n WHERE t2b = t1b" + } ] +} + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + EXCEPT ALL + SELECT t3a + FROM t3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (t2b#x = outer(t1b#x))\n+- SubqueryAlias t2\n +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x])\n +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x]\n +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n +- SubqueryAlias t2\n +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 104, + "fragment" : "SELECT t2a\n FROM t2\n WHERE t2b = t1b" + } ] +} + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + EXCEPT DISTINCT + SELECT t3a + FROM t3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (t2b#x = outer(t1b#x))\n+- SubqueryAlias t2\n +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x])\n +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x]\n +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n +- SubqueryAlias t2\n +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 104, + "fragment" : "SELECT t2a\n FROM t2\n WHERE t2b = t1b" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out new file mode 100644 index 0000000000000..c6eceff7598ca --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out @@ -0,0 +1,743 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query analysis +CreateViewCommand `t1`, select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query analysis +CreateViewCommand `t2`, select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i), false, false, LocalTempView, true + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query analysis +CreateViewCommand `t3`, select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i), false, false, LocalTempView, true + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1a = "val1a") +SELECT t1a, + t1b, + t1c, + t1d, + t1h +FROM t1 +WHERE t1b IN (SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b > 0) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte1 +: +- Project [t1a#x, t1b#x] +: +- Filter (t1a#x = val1a) +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] + +- Filter t1b#x IN (list#x []) + : +- Project [t1b#x] + : +- Filter (cast(t1b#x as int) > 0) + : +- SubqueryAlias cte1 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +WITH cte1 AS +( + SELECT t1a, + t1b + FROM t1) +SELECT count(distinct(t1a)), t1b, t1c +FROM t1 +WHERE t1b IN + ( + SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b > 0 + UNION + SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b > 5 + UNION ALL + SELECT cte1.t1b + FROM cte1 + INTERSECT + SELECT cte1.t1b + FROM cte1 + UNION + SELECT cte1.t1b + FROM cte1 ) +GROUP BY t1a, t1b, t1c +HAVING t1c IS NOT NULL +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte1 +: +- Project [t1a#x, t1b#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Filter isnotnull(t1c#x) + +- Aggregate [t1a#x, t1b#x, t1c#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x, t1c#x] + +- Filter t1b#x IN (list#x []) + : +- Distinct + : +- Union false, false + : :- Union false, false + : : :- Distinct + : : : +- Union false, false + : : : :- Project [t1b#x] + : : : : +- Filter (cast(t1b#x as int) > 0) + : : : : +- SubqueryAlias cte1 + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : : : +- Project [t1b#x AS t1b#x] + : : : +- Project [t1b#x] + : : : +- Filter (cast(t1b#x as int) > 5) + : : : +- SubqueryAlias cte1 + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : : +- Project [t1b#x AS t1b#x] + : : +- Intersect false + : : :- Project [t1b#x] + : : : +- SubqueryAlias cte1 + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : : +- Project [t1b#x] + : : +- SubqueryAlias cte1 + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : +- Project [t1b#x AS t1b#x] + : +- Project [t1b#x] + : +- SubqueryAlias cte1 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +WITH cte1 AS +( + SELECT t1a, + t1b, + t1c, + t1d, + t1e + FROM t1) +SELECT t1a, + t1b, + t1c, + t1h +FROM t1 +WHERE t1c IN + ( + SELECT cte1.t1c + FROM cte1 + JOIN cte1 cte2 + on cte1.t1b > cte2.t1b + FULL OUTER JOIN cte1 cte3 + ON cte1.t1c = cte3.t1c + LEFT JOIN cte1 cte4 + ON cte1.t1d = cte4.t1d + INNER JOIN cte1 cte5 + ON cte1.t1b < cte5.t1b + LEFT OUTER JOIN cte1 cte6 + ON cte1.t1d > cte6.t1d) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte1 +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Project [t1a#x, t1b#x, t1c#x, t1h#x] + +- Filter t1c#x IN (list#x []) + : +- Project [t1c#x] + : +- Join LeftOuter, (t1d#xL > t1d#xL) + : :- Join Inner, (t1b#x < t1b#x) + : : :- Join LeftOuter, (t1d#xL = t1d#xL) + : : : :- Join FullOuter, (t1c#x = t1c#x) + : : : : :- Join Inner, (t1b#x > t1b#x) + : : : : : :- SubqueryAlias cte1 + : : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] + : : : : : +- SubqueryAlias cte2 + : : : : : +- SubqueryAlias cte1 + : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] + : : : : +- SubqueryAlias cte3 + : : : : +- SubqueryAlias cte1 + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] + : : : +- SubqueryAlias cte4 + : : : +- SubqueryAlias cte1 + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] + : : +- SubqueryAlias cte5 + : : +- SubqueryAlias cte1 + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] + : +- SubqueryAlias cte6 + : +- SubqueryAlias cte1 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1b IN (SELECT t2b + FROM t2 + RIGHT JOIN t1 + ON t1c = t2c + LEFT JOIN t3 + ON t2d = t3d) + AND t1a = "val1b") +SELECT * +FROM (SELECT * + FROM cte1 + JOIN cte1 cte2 + ON cte1.t1b > 5 + AND cte1.t1a = cte2.t1a + FULL OUTER JOIN cte1 cte3 + ON cte1.t1a = cte3.t1a + INNER JOIN cte1 cte4 + ON cte1.t1b = cte4.t1b) s +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte1 +: +- Project [t1a#x, t1b#x] +: +- Filter (t1b#x IN (list#x []) AND (t1a#x = val1b)) +: : +- Project [t2b#x] +: : +- Join LeftOuter, (t2d#xL = t3d#xL) +: : :- Join RightOuter, (t1c#x = t2c#x) +: : : :- SubqueryAlias t2 +: : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : : : +- SubqueryAlias t2 +: : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : : +- SubqueryAlias t1 +: : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: : : +- SubqueryAlias t1 +: : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: : +- SubqueryAlias t3 +: : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) +: : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] +: : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: : +- SubqueryAlias t3 +: : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Project [t1a#x, t1b#x, t1a#x, t1b#x, t1a#x, t1b#x, t1a#x, t1b#x] + +- SubqueryAlias s + +- Project [t1a#x, t1b#x, t1a#x, t1b#x, t1a#x, t1b#x, t1a#x, t1b#x] + +- Join Inner, (t1b#x = t1b#x) + :- Join FullOuter, (t1a#x = t1a#x) + : :- Join Inner, ((cast(t1b#x as int) > 5) AND (t1a#x = t1a#x)) + : : :- SubqueryAlias cte1 + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : : +- SubqueryAlias cte2 + : : +- SubqueryAlias cte1 + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : +- SubqueryAlias cte3 + : +- SubqueryAlias cte1 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + +- SubqueryAlias cte4 + +- SubqueryAlias cte1 + +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + + +-- !query +WITH cte1 AS +( + SELECT t1a, + t1b, + t1h + FROM t1 + WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1b < t2b)) +SELECT Count(DISTINCT t1a), + t1b +FROM ( + SELECT cte1.t1a, + cte1.t1b + FROM cte1 + JOIN cte1 cte2 + on cte1.t1h >= cte2.t1h) s +WHERE t1b IN + ( + SELECT t1b + FROM t1) +GROUP BY t1b +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte1 +: +- Project [t1a#x, t1b#x, t1h#x] +: +- Filter t1a#x IN (list#x [t1b#x]) +: : +- Project [t2a#x] +: : +- Filter (outer(t1b#x) < t2b#x) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Aggregate [t1b#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x] + +- Filter t1b#x IN (list#x []) + : +- Project [t1b#x] + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias s + +- Project [t1a#x, t1b#x] + +- Join Inner, (t1h#x >= t1h#x) + :- SubqueryAlias cte1 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x] + +- SubqueryAlias cte2 + +- SubqueryAlias cte1 + +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x] + + +-- !query +WITH cte1 AS +( + SELECT t1a, + t1b, + t1c + FROM t1 + WHERE t1b IN + ( + SELECT t2b + FROM t2 FULL OUTER JOIN T3 on t2a = t3a + WHERE t1c = t2c) AND + t1a = "val1b") +SELECT * +FROM ( + SELECT * + FROM cte1 + INNER JOIN cte1 cte2 ON cte1.t1a = cte2.t1a + RIGHT OUTER JOIN cte1 cte3 ON cte1.t1b = cte3.t1b + LEFT OUTER JOIN cte1 cte4 ON cte1.t1c = cte4.t1c + ) s +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte1 +: +- Project [t1a#x, t1b#x, t1c#x] +: +- Filter (t1b#x IN (list#x [t1c#x]) AND (t1a#x = val1b)) +: : +- Project [t2b#x] +: : +- Filter (outer(t1c#x) = t2c#x) +: : +- Join FullOuter, (t2a#x = t3a#x) +: : :- SubqueryAlias t2 +: : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : : +- SubqueryAlias t2 +: : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- SubqueryAlias t3 +: : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) +: : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] +: : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: : +- SubqueryAlias t3 +: : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Project [t1a#x, t1b#x, t1c#x, t1a#x, t1b#x, t1c#x, t1a#x, t1b#x, t1c#x, t1a#x, t1b#x, t1c#x] + +- SubqueryAlias s + +- Project [t1a#x, t1b#x, t1c#x, t1a#x, t1b#x, t1c#x, t1a#x, t1b#x, t1c#x, t1a#x, t1b#x, t1c#x] + +- Join LeftOuter, (t1c#x = t1c#x) + :- Join RightOuter, (t1b#x = t1b#x) + : :- Join Inner, (t1a#x = t1a#x) + : : :- SubqueryAlias cte1 + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x] + : : +- SubqueryAlias cte2 + : : +- SubqueryAlias cte1 + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x] + : +- SubqueryAlias cte3 + : +- SubqueryAlias cte1 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x] + +- SubqueryAlias cte4 + +- SubqueryAlias cte1 + +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x] + + +-- !query +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1c = t2c)) +SELECT Count(DISTINCT( s.t1a )), + s.t1b +FROM (SELECT cte1.t1a, + cte1.t1b + FROM cte1 + RIGHT OUTER JOIN cte1 cte2 + ON cte1.t1a = cte2.t1a) s +GROUP BY s.t1b +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte1 +: +- Project [t1a#x, t1b#x] +: +- Filter t1b#x IN (list#x [t1c#x]) +: : +- Project [t2b#x] +: : +- Filter (outer(t1c#x) = t2c#x) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Aggregate [t1b#x], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x] + +- SubqueryAlias s + +- Project [t1a#x, t1b#x] + +- Join RightOuter, (t1a#x = t1a#x) + :- SubqueryAlias cte1 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + +- SubqueryAlias cte2 + +- SubqueryAlias cte1 + +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + + +-- !query +WITH cte1 AS +( + SELECT t1a, + t1b + FROM t1 + WHERE t1b IN + ( + SELECT t2b + FROM t2 + WHERE t1c = t2c)) +SELECT DISTINCT(s.t1b) +FROM ( + SELECT cte1.t1b + FROM cte1 + LEFT OUTER JOIN cte1 cte2 + ON cte1.t1b = cte2.t1b) s +WHERE s.t1b IN + ( + SELECT t1.t1b + FROM t1 INNER + JOIN cte1 + ON t1.t1a = cte1.t1a) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte1 +: +- Project [t1a#x, t1b#x] +: +- Filter t1b#x IN (list#x [t1c#x]) +: : +- Project [t2b#x] +: : +- Filter (outer(t1c#x) = t2c#x) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Distinct + +- Project [t1b#x] + +- Filter t1b#x IN (list#x []) + : +- Project [t1b#x] + : +- Join Inner, (t1a#x = t1a#x) + : :- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias cte1 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + +- SubqueryAlias s + +- Project [t1b#x] + +- Join LeftOuter, (t1b#x = t1b#x) + :- SubqueryAlias cte1 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + +- SubqueryAlias cte2 + +- SubqueryAlias cte1 + +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + + +-- !query +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1a = "val1d") +SELECT t1a, + t1b, + t1c, + t1h +FROM t1 +WHERE t1b NOT IN (SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b < 0) AND + t1c > 10 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte1 +: +- Project [t1a#x, t1b#x] +: +- Filter (t1a#x = val1d) +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Project [t1a#x, t1b#x, t1c#x, t1h#x] + +- Filter (NOT t1b#x IN (list#x []) AND (t1c#x > 10)) + : +- Project [t1b#x] + : +- Filter (cast(t1b#x as int) < 0) + : +- SubqueryAlias cte1 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +WITH cte1 AS +( + SELECT t1a, + t1b, + t1c, + t1d, + t1h + FROM t1 + WHERE t1d NOT IN + ( + SELECT t2d + FROM t2 + FULL OUTER JOIN t3 ON t2a = t3a + JOIN t1 on t1b = t2b)) +SELECT t1a, + t1b, + t1c, + t1d, + t1h +FROM t1 +WHERE t1b NOT IN + ( + SELECT cte1.t1b + FROM cte1 INNER + JOIN cte1 cte2 ON cte1.t1a = cte2.t1a + RIGHT JOIN cte1 cte3 ON cte1.t1b = cte3.t1b + JOIN cte1 cte4 ON cte1.t1c = cte4.t1c) AND + t1c IS NOT NULL +ORDER BY t1c DESC +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias cte1 +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] +: +- Filter NOT t1d#xL IN (list#x []) +: : +- Project [t2d#xL] +: : +- Join Inner, (t1b#x = t2b#x) +: : :- Join FullOuter, (t2a#x = t3a#x) +: : : :- SubqueryAlias t2 +: : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : : : +- SubqueryAlias t2 +: : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : : +- SubqueryAlias t3 +: : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) +: : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] +: : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: : : +- SubqueryAlias t3 +: : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: : +- SubqueryAlias t1 +: : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Sort [t1c#x DESC NULLS LAST], true + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] + +- Filter (NOT t1b#x IN (list#x []) AND isnotnull(t1c#x)) + : +- Project [t1b#x] + : +- Join Inner, (t1c#x = t1c#x) + : :- Join RightOuter, (t1b#x = t1b#x) + : : :- Join Inner, (t1a#x = t1a#x) + : : : :- SubqueryAlias cte1 + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] + : : : +- SubqueryAlias cte2 + : : : +- SubqueryAlias cte1 + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] + : : +- SubqueryAlias cte3 + : : +- SubqueryAlias cte1 + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] + : +- SubqueryAlias cte4 + : +- SubqueryAlias cte1 + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/nested-not-in.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/nested-not-in.sql.out new file mode 100644 index 0000000000000..426d3ff5e6a86 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/nested-not-in.sql.out @@ -0,0 +1,546 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", 10), + (200, "emp 2", NULL), + (300, "emp 3", 20), + (400, "emp 4", 30), + (500, "emp 5", NULL), + (600, "emp 6", 100), + (800, "emp 8", 70) +AS EMP(id, emp_name, dept_id) +-- !query analysis +CreateViewCommand `EMP`, SELECT * FROM VALUES + (100, "emp 1", 10), + (200, "emp 2", NULL), + (300, "emp 3", 20), + (400, "emp 4", 30), + (500, "emp 5", NULL), + (600, "emp 6", 100), + (800, "emp 8", 70) +AS EMP(id, emp_name, dept_id), false, false, LocalTempView, true + +- Project [id#x, emp_name#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, dept_id#x] + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query analysis +CreateViewCommand `DEPT`, SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state), false, false, LocalTempView, true + +- Project [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias DEPT + +- LocalRelation [dept_id#x, dept_name#x, state#x] + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query analysis +CreateViewCommand `BONUS`, SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt), false, false, LocalTempView, true + +- Project [emp_name#x, bonus_amt#x] + +- SubqueryAlias BONUS + +- LocalRelation [emp_name#x, bonus_amt#x] + + +-- !query +CREATE TEMPORARY VIEW ADDRESS AS SELECT * FROM VALUES + (100, "emp 1", "addr1"), + (200, null, "addr2"), + (null, "emp 3", "addr3"), + (null, null, "addr4"), + (600, "emp 6", "addr6"), + (800, "emp 8", "addr8") +AS ADDRESS(id, emp_name, address) +-- !query analysis +CreateViewCommand `ADDRESS`, SELECT * FROM VALUES + (100, "emp 1", "addr1"), + (200, null, "addr2"), + (null, "emp 3", "addr3"), + (null, null, "addr4"), + (600, "emp 6", "addr6"), + (800, "emp 8", "addr8") +AS ADDRESS(id, emp_name, address), false, false, LocalTempView, true + +- Project [id#x, emp_name#x, address#x] + +- SubqueryAlias ADDRESS + +- LocalRelation [id#x, emp_name#x, address#x] + + +-- !query +CREATE TEMPORARY VIEW S1 AS SELECT * FROM VALUES + (null, null), (5, 5), (8, 8), (11, 11) AS s1(a, b) +-- !query analysis +CreateViewCommand `S1`, SELECT * FROM VALUES + (null, null), (5, 5), (8, 8), (11, 11) AS s1(a, b), false, false, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias s1 + +- LocalRelation [a#x, b#x] + + +-- !query +CREATE TEMPORARY VIEW S2 AS SELECT * FROM VALUES + (7, 7), (8, 8), (11, 11), (null, null) AS s2(c, d) +-- !query analysis +CreateViewCommand `S2`, SELECT * FROM VALUES + (7, 7), (8, 8), (11, 11), (null, null) AS s2(c, d), false, false, LocalTempView, true + +- Project [c#x, d#x] + +- SubqueryAlias s2 + +- LocalRelation [c#x, d#x] + + +-- !query +SELECT id, + dept_id +FROM emp +WHERE id = 600 + OR id = 500 + OR dept_id NOT IN (SELECT dept_id + FROM emp) +-- !query analysis +Project [id#x, dept_id#x] ++- Filter (((id#x = 600) OR (id#x = 500)) OR NOT dept_id#x IN (list#x [])) + : +- Project [dept_id#x] + : +- SubqueryAlias emp + : +- View (`EMP`, [id#x,emp_name#x,dept_id#x]) + : +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(dept_id#x as int) AS dept_id#x] + : +- Project [id#x, emp_name#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, dept_id#x] + +- SubqueryAlias emp + +- View (`EMP`, [id#x,emp_name#x,dept_id#x]) + +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(dept_id#x as int) AS dept_id#x] + +- Project [id#x, emp_name#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, dept_id#x] + + +-- !query +SELECT id, + dept_id +FROM emp +WHERE id = 800 + OR (dept_id IS NOT NULL + AND dept_id NOT IN (SELECT dept_id + FROM emp)) +-- !query analysis +Project [id#x, dept_id#x] ++- Filter ((id#x = 800) OR (isnotnull(dept_id#x) AND NOT dept_id#x IN (list#x []))) + : +- Project [dept_id#x] + : +- SubqueryAlias emp + : +- View (`EMP`, [id#x,emp_name#x,dept_id#x]) + : +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(dept_id#x as int) AS dept_id#x] + : +- Project [id#x, emp_name#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, dept_id#x] + +- SubqueryAlias emp + +- View (`EMP`, [id#x,emp_name#x,dept_id#x]) + +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(dept_id#x as int) AS dept_id#x] + +- Project [id#x, emp_name#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, dept_id#x] + + +-- !query +SELECT id, + dept_id +FROM emp +WHERE id = 100 + OR dept_id NOT IN (SELECT dept_id + FROM emp + WHERE dept_id IS NOT NULL) +-- !query analysis +Project [id#x, dept_id#x] ++- Filter ((id#x = 100) OR NOT dept_id#x IN (list#x [])) + : +- Project [dept_id#x] + : +- Filter isnotnull(dept_id#x) + : +- SubqueryAlias emp + : +- View (`EMP`, [id#x,emp_name#x,dept_id#x]) + : +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(dept_id#x as int) AS dept_id#x] + : +- Project [id#x, emp_name#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, dept_id#x] + +- SubqueryAlias emp + +- View (`EMP`, [id#x,emp_name#x,dept_id#x]) + +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(dept_id#x as int) AS dept_id#x] + +- Project [id#x, emp_name#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, dept_id#x] + + +-- !query +SELECT id, + dept_id +FROM emp +WHERE id = 200 + OR (dept_id IS NOT NULL + AND dept_id + 100 NOT IN (SELECT dept_id + FROM emp + WHERE dept_id IS NOT NULL)) +-- !query analysis +Project [id#x, dept_id#x] ++- Filter ((id#x = 200) OR (isnotnull(dept_id#x) AND NOT (dept_id#x + 100) IN (list#x []))) + : +- Project [dept_id#x] + : +- Filter isnotnull(dept_id#x) + : +- SubqueryAlias emp + : +- View (`EMP`, [id#x,emp_name#x,dept_id#x]) + : +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(dept_id#x as int) AS dept_id#x] + : +- Project [id#x, emp_name#x, dept_id#x] + : +- SubqueryAlias EMP + : +- LocalRelation [id#x, emp_name#x, dept_id#x] + +- SubqueryAlias emp + +- View (`EMP`, [id#x,emp_name#x,dept_id#x]) + +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(dept_id#x as int) AS dept_id#x] + +- Project [id#x, emp_name#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, dept_id#x] + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE emp_name IN (SELECT emp_name + FROM bonus) + OR (dept_id IS NOT NULL + AND dept_id NOT IN (SELECT dept_id + FROM dept)) +-- !query analysis +Project [id#x, dept_id#x, emp_name#x] ++- Filter (emp_name#x IN (list#x []) OR (isnotnull(dept_id#x) AND NOT dept_id#x IN (list#x []))) + : :- Project [emp_name#x] + : : +- SubqueryAlias bonus + : : +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + : : +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + : : +- Project [emp_name#x, bonus_amt#x] + : : +- SubqueryAlias BONUS + : : +- LocalRelation [emp_name#x, bonus_amt#x] + : +- Project [dept_id#x] + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias emp + +- View (`EMP`, [id#x,emp_name#x,dept_id#x]) + +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(dept_id#x as int) AS dept_id#x] + +- Project [id#x, emp_name#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, dept_id#x] + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE EXISTS (SELECT emp_name + FROM bonus + WHERE emp.emp_name = bonus.emp_name) + OR (dept_id IS NOT NULL + AND dept_id NOT IN (SELECT dept_id + FROM dept)) +-- !query analysis +Project [id#x, dept_id#x, emp_name#x] ++- Filter (exists#x [emp_name#x] OR (isnotnull(dept_id#x) AND NOT dept_id#x IN (list#x []))) + : :- Project [emp_name#x] + : : +- Filter (outer(emp_name#x) = emp_name#x) + : : +- SubqueryAlias bonus + : : +- View (`BONUS`, [emp_name#x,bonus_amt#x]) + : : +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] + : : +- Project [emp_name#x, bonus_amt#x] + : : +- SubqueryAlias BONUS + : : +- LocalRelation [emp_name#x, bonus_amt#x] + : +- Project [dept_id#x] + : +- SubqueryAlias dept + : +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) + : +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x] + : +- Project [dept_id#x, dept_name#x, state#x] + : +- SubqueryAlias DEPT + : +- LocalRelation [dept_id#x, dept_name#x, state#x] + +- SubqueryAlias emp + +- View (`EMP`, [id#x,emp_name#x,dept_id#x]) + +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(dept_id#x as int) AS dept_id#x] + +- Project [id#x, emp_name#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, dept_id#x] + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE dept_id = 10 +OR (id, emp_name) NOT IN (SELECT id, emp_name FROM address) +-- !query analysis +Project [id#x, dept_id#x, emp_name#x] ++- Filter ((dept_id#x = 10) OR NOT named_struct(id, id#x, emp_name, emp_name#x) IN (list#x [])) + : +- Project [id#x, emp_name#x] + : +- SubqueryAlias address + : +- View (`ADDRESS`, [id#x,emp_name#x,address#x]) + : +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(address#x as string) AS address#x] + : +- Project [id#x, emp_name#x, address#x] + : +- SubqueryAlias ADDRESS + : +- LocalRelation [id#x, emp_name#x, address#x] + +- SubqueryAlias emp + +- View (`EMP`, [id#x,emp_name#x,dept_id#x]) + +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(dept_id#x as int) AS dept_id#x] + +- Project [id#x, emp_name#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, dept_id#x] + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE dept_id = 10 + OR (( id, emp_name ) NOT IN (SELECT id, + emp_name + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL) + AND id > 400 ) +-- !query analysis +Project [id#x, dept_id#x, emp_name#x] ++- Filter ((dept_id#x = 10) OR (NOT named_struct(id, id#x, emp_name, emp_name#x) IN (list#x []) AND (id#x > 400))) + : +- Project [id#x, emp_name#x] + : +- Filter (isnotnull(id#x) AND isnotnull(emp_name#x)) + : +- SubqueryAlias address + : +- View (`ADDRESS`, [id#x,emp_name#x,address#x]) + : +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(address#x as string) AS address#x] + : +- Project [id#x, emp_name#x, address#x] + : +- SubqueryAlias ADDRESS + : +- LocalRelation [id#x, emp_name#x, address#x] + +- SubqueryAlias emp + +- View (`EMP`, [id#x,emp_name#x,dept_id#x]) + +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(dept_id#x as int) AS dept_id#x] + +- Project [id#x, emp_name#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, dept_id#x] + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE dept_id = 10 + OR emp_name NOT IN (SELECT emp_name + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL + AND emp.id = address.id) +-- !query analysis +Project [id#x, dept_id#x, emp_name#x] ++- Filter ((dept_id#x = 10) OR NOT emp_name#x IN (list#x [id#x])) + : +- Project [emp_name#x] + : +- Filter ((isnotnull(id#x) AND isnotnull(emp_name#x)) AND (outer(id#x) = id#x)) + : +- SubqueryAlias address + : +- View (`ADDRESS`, [id#x,emp_name#x,address#x]) + : +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(address#x as string) AS address#x] + : +- Project [id#x, emp_name#x, address#x] + : +- SubqueryAlias ADDRESS + : +- LocalRelation [id#x, emp_name#x, address#x] + +- SubqueryAlias emp + +- View (`EMP`, [id#x,emp_name#x,dept_id#x]) + +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(dept_id#x as int) AS dept_id#x] + +- Project [id#x, emp_name#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, dept_id#x] + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE id NOT IN (SELECT id + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL + AND id >= 400) + OR emp_name NOT IN (SELECT emp_name + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL + AND emp.id = address.id + AND id < 400) +-- !query analysis +Project [id#x, dept_id#x, emp_name#x] ++- Filter (NOT id#x IN (list#x []) OR NOT emp_name#x IN (list#x [id#x])) + : :- Project [id#x] + : : +- Filter ((isnotnull(id#x) AND isnotnull(emp_name#x)) AND (id#x >= 400)) + : : +- SubqueryAlias address + : : +- View (`ADDRESS`, [id#x,emp_name#x,address#x]) + : : +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(address#x as string) AS address#x] + : : +- Project [id#x, emp_name#x, address#x] + : : +- SubqueryAlias ADDRESS + : : +- LocalRelation [id#x, emp_name#x, address#x] + : +- Project [emp_name#x] + : +- Filter ((isnotnull(id#x) AND isnotnull(emp_name#x)) AND ((outer(id#x) = id#x) AND (id#x < 400))) + : +- SubqueryAlias address + : +- View (`ADDRESS`, [id#x,emp_name#x,address#x]) + : +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(address#x as string) AS address#x] + : +- Project [id#x, emp_name#x, address#x] + : +- SubqueryAlias ADDRESS + : +- LocalRelation [id#x, emp_name#x, address#x] + +- SubqueryAlias emp + +- View (`EMP`, [id#x,emp_name#x,dept_id#x]) + +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(dept_id#x as int) AS dept_id#x] + +- Project [id#x, emp_name#x, dept_id#x] + +- SubqueryAlias EMP + +- LocalRelation [id#x, emp_name#x, dept_id#x] + + +-- !query +SELECT * +FROM s1 +WHERE NOT (a NOT IN (SELECT c + FROM s2)) +-- !query analysis +Project [a#x, b#x] ++- Filter NOT NOT a#x IN (list#x []) + : +- Project [c#x] + : +- SubqueryAlias s2 + : +- View (`S2`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s2 + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias s1 + +- View (`S1`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias s1 + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM s1 +WHERE NOT (a > 5 + OR a IN (SELECT c + FROM s2)) +-- !query analysis +Project [a#x, b#x] ++- Filter NOT ((a#x > 5) OR a#x IN (list#x [])) + : +- Project [c#x] + : +- SubqueryAlias s2 + : +- View (`S2`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s2 + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias s1 + +- View (`S1`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias s1 + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM s1 +WHERE NOT (a > 5 + OR a NOT IN (SELECT c + FROM s2)) +-- !query analysis +Project [a#x, b#x] ++- Filter NOT ((a#x > 5) OR NOT a#x IN (list#x [])) + : +- Project [c#x] + : +- SubqueryAlias s2 + : +- View (`S2`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s2 + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias s1 + +- View (`S1`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias s1 + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM s1 +WHERE NOT (a > 5 + AND a IN (SELECT c + FROM s2)) +-- !query analysis +Project [a#x, b#x] ++- Filter NOT ((a#x > 5) AND a#x IN (list#x [])) + : +- Project [c#x] + : +- SubqueryAlias s2 + : +- View (`S2`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s2 + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias s1 + +- View (`S1`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias s1 + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM s1 +WHERE NOT (a > 5 + AND a NOT IN (SELECT c + FROM s2)) +-- !query analysis +Project [a#x, b#x] ++- Filter NOT ((a#x > 5) AND NOT a#x IN (list#x [])) + : +- Project [c#x] + : +- SubqueryAlias s2 + : +- View (`S2`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as int) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s2 + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias s1 + +- View (`S1`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias s1 + +- LocalRelation [a#x, b#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-group-by.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-group-by.sql.out new file mode 100644 index 0000000000000..cd516fc99a94f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-group-by.sql.out @@ -0,0 +1,268 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query analysis +CreateViewCommand `t1`, select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query analysis +CreateViewCommand `t2`, select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i), false, false, LocalTempView, true + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query analysis +CreateViewCommand `t3`, select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i), false, false, LocalTempView, true + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT t1a, + Avg(t1b) +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2) +GROUP BY t1a +-- !query analysis +Aggregate [t1a#x], [t1a#x, avg(t1b#x) AS avg(t1b)#x] ++- Filter NOT t1a#x IN (list#x []) + : +- Project [t2a#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + WHERE t1h < t2h) +GROUP BY t1a +-- !query analysis +Aggregate [t1a#x], [t1a#x, sum(distinct t1b#x) AS sum(DISTINCT t1b)#xL] ++- Filter NOT t1d#xL IN (list#x [t1h#x]) + : +- Project [t2d#xL] + : +- Filter (outer(t1h#x) < t2h#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT Count(*) +FROM (SELECT * + FROM t2 + WHERE t2a NOT IN (SELECT t3a + FROM t3 + WHERE t3h != t2h)) t2 +WHERE t2b NOT IN (SELECT Min(t2b) + FROM t2 + WHERE t2b = t2b + GROUP BY t2c) +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter NOT t2b#x IN (list#x []) + : +- Aggregate [t2c#x], [min(t2b#x) AS min(t2b)#x] + : +- Filter (t2b#x = t2b#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Filter NOT t2a#x IN (list#x [t2h#x]) + : +- Project [t3a#x] + : +- Filter NOT (t3h#x = outer(t2h#x)) + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t2 + +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +SELECT t1a, + max(t1b) +FROM t1 +WHERE t1c NOT IN (SELECT Max(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2a) +GROUP BY t1a +-- !query analysis +Aggregate [t1a#x], [t1a#x, max(t1b#x) AS max(t1b)#x] ++- Filter NOT t1c#x IN (list#x [t1a#x]) + : +- Project [cast(max(t2b)#x as int) AS max(t2b)#x] + : +- Aggregate [t2a#x], [max(t2b#x) AS max(t2b)#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t2a NOT IN (SELECT Min(t3a) + FROM t3 + WHERE t3a = t2a + GROUP BY t3b) order by t2a) +-- !query analysis +Project [t1a#x, t1b#x] ++- Filter t1c#x IN (list#x []) + : +- Project [cast(t2b#x as int) AS t2b#x] + : +- Project [t2b#x] + : +- Sort [t2a#x ASC NULLS FIRST], true + : +- Project [t2b#x, t2a#x] + : +- Filter NOT t2a#x IN (list#x [t2a#x]) + : : +- Aggregate [t3b#x], [min(t3a#x) AS min(t3a)#x] + : : +- Filter (t3a#x = outer(t2a#x)) + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-joins.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-joins.sql.out new file mode 100644 index 0000000000000..afcbb82a6a694 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-joins.sql.out @@ -0,0 +1,429 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query analysis +CreateViewCommand `t1`, select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query analysis +CreateViewCommand `t2`, select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i), false, false, LocalTempView, true + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query analysis +CreateViewCommand `t3`, select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i), false, false, LocalTempView, true + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT t1a, + t1b, + t1c, + t3a, + t3b, + t3c +FROM t1 + JOIN t3 +WHERE t1a NOT IN (SELECT t2a + FROM t2) + AND t1b = t3b +-- !query analysis +Project [t1a#x, t1b#x, t1c#x, t3a#x, t3b#x, t3c#x] ++- Filter (NOT t1a#x IN (list#x []) AND (t1b#x = t3b#x)) + : +- Project [t2a#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Join Inner + :- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t3 + +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT t1a, + t1b, + t1c, + count(distinct(t3a)), + t3b, + t3c +FROM t1 +FULL OUTER JOIN t3 on t1b != t3b +RIGHT JOIN t2 on t1c = t2c +where t1a NOT IN + ( + SELECT t2a + FROM t2 + WHERE t2c NOT IN + ( + SELECT t1c + FROM t1 + WHERE t1a = t2a)) +AND t1b != t3b +AND t1d = t2d +GROUP BY t1a, t1b, t1c, t3a, t3b, t3c +HAVING count(distinct(t3a)) >= 1 +ORDER BY t1a, t3b +-- !query analysis +Sort [t1a#x ASC NULLS FIRST, t3b#x ASC NULLS FIRST], true ++- Filter (count(DISTINCT t3a)#xL >= cast(1 as bigint)) + +- Aggregate [t1a#x, t1b#x, t1c#x, t3a#x, t3b#x, t3c#x], [t1a#x, t1b#x, t1c#x, count(distinct t3a#x) AS count(DISTINCT t3a)#xL, t3b#x, t3c#x] + +- Filter ((NOT t1a#x IN (list#x []) AND NOT (t1b#x = t3b#x)) AND (t1d#xL = t2d#xL)) + : +- Project [t2a#x] + : +- Filter NOT t2c#x IN (list#x [t2a#x]) + : : +- Project [t1c#x] + : : +- Filter (t1a#x = outer(t2a#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Join RightOuter, (t1c#x = t2c#x) + :- Join FullOuter, NOT (t1b#x = t3b#x) + : :- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t2 + +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +SELECT t1a, + t1b, + t1c, + t1d, + t1h +FROM t1 +WHERE t1a NOT IN + ( + SELECT t2a + FROM t2 + LEFT JOIN t3 on t2b = t3b + WHERE t1d = t2d + ) +AND t1d NOT IN + ( + SELECT t2d + FROM t2 + RIGHT JOIN t1 on t2e = t1e + WHERE t1a = t2a) +-- !query analysis +Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] ++- Filter (NOT t1a#x IN (list#x [t1d#xL]) AND NOT t1d#xL IN (list#x [])) + : :- Project [t2a#x] + : : +- Filter (outer(t1d#xL) = t2d#xL) + : : +- Join LeftOuter, (t2b#x = t3b#x) + : : :- SubqueryAlias t2 + : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- SubqueryAlias t2 + : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- Project [t2d#xL] + : +- Filter (t1a#x = t2a#x) + : +- Join RightOuter, (t2e#x = t1e#x) + : :- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + JOIN t1 + WHERE t2b <> t1b) +GROUP BY t1b, + t1c, + t1d +HAVING t1d NOT IN (SELECT t2d + FROM t2 + WHERE t1d = t2d) +ORDER BY t1b DESC, t1d ASC +-- !query analysis +Sort [t1b#x DESC NULLS LAST, t1d#xL ASC NULLS FIRST], true ++- Filter NOT t1d#xL IN (list#x [t1d#xL]) + : +- Project [t2d#xL] + : +- Filter (outer(t1d#xL) = t2d#xL) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Aggregate [t1b#x, t1c#x, t1d#xL], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x, t1c#x, t1d#xL] + +- Filter NOT t1a#x IN (list#x []) + : +- Project [t2a#x] + : +- Filter NOT (t2b#x = t1b#x) + : +- Join Inner + : :- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT COUNT(DISTINCT(t1a)), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a NOT IN + ( + SELECT t2a + FROM t2 INNER + JOIN t1 ON t1a = t2a) +GROUP BY t1b, + t1c, + t1d +HAVING t1b < sum(t1c) +-- !query analysis +Project [count(DISTINCT t1a)#xL, t1b#x, t1c#x, t1d#xL] ++- Filter (cast(t1b#x as bigint) < sum(t1c#x)#xL) + +- Aggregate [t1b#x, t1c#x, t1d#xL], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x, t1c#x, t1d#xL, sum(t1c#x) AS sum(t1c#x)#xL] + +- Filter NOT t1a#x IN (list#x []) + : +- Project [t2a#x] + : +- Join Inner, (t1a#x = t2a#x) + : :- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT COUNT(DISTINCT(t1a)), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a NOT IN + ( + SELECT t2a + FROM t2 INNER + JOIN t1 + ON t1a = t2a) +AND t1d NOT IN + ( + SELECT t2d + FROM t2 + INNER JOIN t3 + ON t2b = t3b ) +GROUP BY t1b, + t1c, + t1d +HAVING t1b < sum(t1c) +-- !query analysis +Project [count(DISTINCT t1a)#xL, t1b#x, t1c#x, t1d#xL] ++- Filter (cast(t1b#x as bigint) < sum(t1c#x)#xL) + +- Aggregate [t1b#x, t1c#x, t1d#xL], [count(distinct t1a#x) AS count(DISTINCT t1a)#xL, t1b#x, t1c#x, t1d#xL, sum(t1c#x) AS sum(t1c#x)#xL] + +- Filter (NOT t1a#x IN (list#x []) AND NOT t1d#xL IN (list#x [])) + : :- Project [t2a#x] + : : +- Join Inner, (t1a#x = t2a#x) + : : :- SubqueryAlias t2 + : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- SubqueryAlias t2 + : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- Project [t2d#xL] + : +- Join Inner, (t2b#x = t3b#x) + : :- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out new file mode 100644 index 0000000000000..587a4801488a4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out @@ -0,0 +1,66 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES + (null, null), + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b) +-- !query analysis +CreateViewCommand `m`, SELECT * FROM VALUES + (null, null), + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b), false, false, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE b = 1.0 -- Matches (null, 1.0) + AND (a, b) NOT IN ((2, 3.0)) +-- !query analysis +Project [a#x, b#x] ++- Filter ((b#x = 1.0) AND NOT named_struct(a, a#x, b, b#x) IN (named_struct(col1, 2, col2, 3.0))) + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE b = 3.0 -- Matches (2, 3.0) + AND (a, b) NOT IN ((2, 3.0)) +-- !query analysis +Project [a#x, b#x] ++- Filter ((b#x = 3.0) AND NOT named_struct(a, a#x, b, b#x) IN (named_struct(col1, 2, col2, 3.0))) + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE b = 5.0 -- Matches (4, 5.0) + AND (a, b) NOT IN ((2, 3.0)) +-- !query analysis +Project [a#x, b#x] ++- Filter ((b#x = 5.0) AND NOT named_struct(a, a#x, b, b#x) IN (named_struct(col1, 2, col2, 3.0))) + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out new file mode 100644 index 0000000000000..369467ff9f8f2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out @@ -0,0 +1,217 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES + (null, null), + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b) +-- !query analysis +CreateViewCommand `m`, SELECT * FROM VALUES + (null, null), + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b), false, false, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +CREATE TEMPORARY VIEW s AS SELECT * FROM VALUES + (null, null), + (0, 1.0), + (2, 3.0), + (4, null) + AS s(c, d) +-- !query analysis +CreateViewCommand `s`, SELECT * FROM VALUES + (null, null), + (0, 1.0), + (2, 3.0), + (4, null) + AS s(c, d), false, false, LocalTempView, true + +- Project [c#x, d#x] + +- SubqueryAlias s + +- LocalRelation [c#x, d#x] + + +-- !query +SELECT * +FROM m +WHERE (a, b) NOT IN (SELECT * + FROM s + WHERE d > 5.0) -- Matches no rows +-- !query analysis +Project [a#x, b#x] ++- Filter NOT named_struct(a, a#x, b, b#x) IN (list#x []) + : +- Project [c#x, d#x] + : +- Filter (d#x > 5.0) + : +- SubqueryAlias s + : +- View (`s`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as decimal(2,1)) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE (a, b) NOT IN (SELECT * + FROM s + WHERE c IS NULL AND d IS NULL) -- Matches only (null, null) +-- !query analysis +Project [a#x, b#x] ++- Filter NOT named_struct(a, a#x, b, b#x) IN (list#x []) + : +- Project [c#x, d#x] + : +- Filter (isnull(c#x) AND isnull(d#x)) + : +- SubqueryAlias s + : +- View (`s`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as decimal(2,1)) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE a IS NULL AND b IS NULL -- Matches only (null, null) + AND (a, b) NOT IN (SELECT * + FROM s + WHERE c IS NOT NULL) -- Matches (0, 1.0), (2, 3.0), (4, null) +-- !query analysis +Project [a#x, b#x] ++- Filter ((isnull(a#x) AND isnull(b#x)) AND NOT named_struct(a, a#x, b, b#x) IN (list#x [])) + : +- Project [c#x, d#x] + : +- Filter isnotnull(c#x) + : +- SubqueryAlias s + : +- View (`s`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as decimal(2,1)) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE b = 1.0 -- Matches (null, 1.0) + AND (a, b) NOT IN (SELECT * + FROM s + WHERE c IS NOT NULL) -- Matches (0, 1.0), (2, 3.0), (4, null) +-- !query analysis +Project [a#x, b#x] ++- Filter ((b#x = 1.0) AND NOT named_struct(a, a#x, b, b#x) IN (list#x [])) + : +- Project [c#x, d#x] + : +- Filter isnotnull(c#x) + : +- SubqueryAlias s + : +- View (`s`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as decimal(2,1)) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE b = 1.0 -- Matches (null, 1.0) + AND (a, b) NOT IN (SELECT * + FROM s + WHERE c = 2) -- Matches (2, 3.0) +-- !query analysis +Project [a#x, b#x] ++- Filter ((b#x = 1.0) AND NOT named_struct(a, a#x, b, b#x) IN (list#x [])) + : +- Project [c#x, d#x] + : +- Filter (c#x = 2) + : +- SubqueryAlias s + : +- View (`s`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as decimal(2,1)) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE b = 3.0 -- Matches (2, 3.0) + AND (a, b) NOT IN (SELECT * + FROM s + WHERE c = 2) -- Matches (2, 3.0) +-- !query analysis +Project [a#x, b#x] ++- Filter ((b#x = 3.0) AND NOT named_struct(a, a#x, b, b#x) IN (list#x [])) + : +- Project [c#x, d#x] + : +- Filter (c#x = 2) + : +- SubqueryAlias s + : +- View (`s`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as decimal(2,1)) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE b = 5.0 -- Matches (4, 5.0) + AND (a, b) NOT IN (SELECT * + FROM s + WHERE c = 2) -- Matches (2, 3.0) +-- !query analysis +Project [a#x, b#x] ++- Filter ((b#x = 5.0) AND NOT named_struct(a, a#x, b, b#x) IN (list#x [])) + : +- Project [c#x, d#x] + : +- Filter (c#x = 2) + : +- SubqueryAlias s + : +- View (`s`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as decimal(2,1)) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out new file mode 100644 index 0000000000000..1d1a9d6c4295e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out @@ -0,0 +1,79 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b) +-- !query analysis +CreateViewCommand `m`, SELECT * FROM VALUES + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b), false, false, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE a NOT IN (null) +-- !query analysis +Project [a#x, b#x] ++- Filter NOT cast(a#x as int) IN (cast(null as int)) + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE b = 1.0 -- Only matches (null, 1.0) + AND a NOT IN (2) +-- !query analysis +Project [a#x, b#x] ++- Filter ((b#x = 1.0) AND NOT a#x IN (2)) + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE b = 3.0 -- Only matches (2, 3.0) + AND a NOT IN (2) +-- !query analysis +Project [a#x, b#x] ++- Filter ((b#x = 3.0) AND NOT a#x IN (2)) + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE b = 3.0 -- Only matches (2, 3.0) + AND a NOT IN (6) +-- !query analysis +Project [a#x, b#x] ++- Filter ((b#x = 3.0) AND NOT a#x IN (6)) + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out new file mode 100644 index 0000000000000..eaa3a2cde6fff --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out @@ -0,0 +1,238 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b) +-- !query analysis +CreateViewCommand `m`, SELECT * FROM VALUES + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b), false, false, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +CREATE TEMPORARY VIEW s AS SELECT * FROM VALUES + (null, 1.0), + (2, 3.0), + (6, 7.0) + AS s(c, d) +-- !query analysis +CreateViewCommand `s`, SELECT * FROM VALUES + (null, 1.0), + (2, 3.0), + (6, 7.0) + AS s(c, d), false, false, LocalTempView, true + +- Project [c#x, d#x] + +- SubqueryAlias s + +- LocalRelation [c#x, d#x] + + +-- !query +SELECT * +FROM m +WHERE a NOT IN (SELECT c + FROM s + WHERE d > 10.0) -- (empty subquery) +-- !query analysis +Project [a#x, b#x] ++- Filter NOT a#x IN (list#x []) + : +- Project [c#x] + : +- Filter (cast(d#x as decimal(3,1)) > 10.0) + : +- SubqueryAlias s + : +- View (`s`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as decimal(2,1)) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE a NOT IN (SELECT c + FROM s + WHERE d = 1.0) -- Only matches (null, 1.0) +-- !query analysis +Project [a#x, b#x] ++- Filter NOT a#x IN (list#x []) + : +- Project [c#x] + : +- Filter (d#x = 1.0) + : +- SubqueryAlias s + : +- View (`s`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as decimal(2,1)) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE b = 1.0 -- Only matches (null, 1.0) + AND a NOT IN (SELECT c + FROM s + WHERE d = 3.0) -- Matches (2, 3.0) +-- !query analysis +Project [a#x, b#x] ++- Filter ((b#x = 1.0) AND NOT a#x IN (list#x [])) + : +- Project [c#x] + : +- Filter (d#x = 3.0) + : +- SubqueryAlias s + : +- View (`s`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as decimal(2,1)) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE b = 3.0 -- Only matches (2, 3.0) + AND a NOT IN (SELECT c + FROM s + WHERE d = 3.0) -- Matches (2, 3.0) +-- !query analysis +Project [a#x, b#x] ++- Filter ((b#x = 3.0) AND NOT a#x IN (list#x [])) + : +- Project [c#x] + : +- Filter (d#x = 3.0) + : +- SubqueryAlias s + : +- View (`s`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as decimal(2,1)) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE b = 3.0 -- Only matches (2, 3.0) + AND a NOT IN (SELECT c + FROM s + WHERE d = 7.0) -- Matches (6, 7.0) +-- !query analysis +Project [a#x, b#x] ++- Filter ((b#x = 3.0) AND NOT a#x IN (list#x [])) + : +- Project [c#x] + : +- Filter (d#x = 7.0) + : +- SubqueryAlias s + : +- View (`s`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as decimal(2,1)) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE a NOT IN (SELECT c + FROM s + WHERE d = b + 10) -- Matches no row +-- !query analysis +Project [a#x, b#x] ++- Filter NOT a#x IN (list#x [b#x]) + : +- Project [c#x] + : +- Filter (cast(d#x as decimal(4,1)) = (outer(b#x) + cast(10 as decimal(2,0)))) + : +- SubqueryAlias s + : +- View (`s`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as decimal(2,1)) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE b = 1.0 -- Only matches (null, 1.0) + AND a NOT IN (SELECT c + FROM s + WHERE d = b + 10) -- Matches no row +-- !query analysis +Project [a#x, b#x] ++- Filter ((b#x = 1.0) AND NOT a#x IN (list#x [b#x])) + : +- Project [c#x] + : +- Filter (cast(d#x as decimal(4,1)) = (outer(b#x) + cast(10 as decimal(2,0)))) + : +- SubqueryAlias s + : +- View (`s`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as decimal(2,1)) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * +FROM m +WHERE b = 3.0 -- Only matches (2, 3.0) + AND a NOT IN (SELECT c + FROM s + WHERE d = b + 10) -- Matches no row +-- !query analysis +Project [a#x, b#x] ++- Filter ((b#x = 3.0) AND NOT a#x IN (list#x [b#x])) + : +- Project [c#x] + : +- Filter (cast(d#x as decimal(4,1)) = (outer(b#x) + cast(10 as decimal(2,0)))) + : +- SubqueryAlias s + : +- View (`s`, [c#x,d#x]) + : +- Project [cast(c#x as int) AS c#x, cast(d#x as decimal(2,1)) AS d#x] + : +- Project [c#x, d#x] + : +- SubqueryAlias s + : +- LocalRelation [c#x, d#x] + +- SubqueryAlias m + +- View (`m`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as decimal(2,1)) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias m + +- LocalRelation [a#x, b#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/simple-in.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/simple-in.sql.out new file mode 100644 index 0000000000000..0aecb80074916 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/simple-in.sql.out @@ -0,0 +1,387 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query analysis +CreateViewCommand `t1`, select * from values + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +create temporary view t2 as select * from values + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query analysis +CreateViewCommand `t2`, select * from values + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i), false, false, LocalTempView, true + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +create temporary view t3 as select * from values + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query analysis +CreateViewCommand `t3`, select * from values + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i), false, false, LocalTempView, true + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2) +-- !query analysis +Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Filter t1a#x IN (list#x []) + : +- Project [t2a#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT * +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +-- !query analysis +Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Filter t1b#x IN (list#x [t1a#x]) + : +- Project [t2b#x] + : +- Filter (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t1a != t2a) +-- !query analysis +Project [t1a#x, t1b#x] ++- Filter t1c#x IN (list#x [t1a#x]) + : +- Project [cast(t2b#x as int) AS t2b#x] + : +- Project [t2b#x] + : +- Filter NOT (outer(t1a#x) = t2a#x) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t1a = t2a + OR t1b > t2b) +-- !query analysis +Project [t1a#x, t1b#x] ++- Filter t1c#x IN (list#x [t1a#x && t1b#x]) + : +- Project [cast(t2b#x as int) AS t2b#x] + : +- Project [t2b#x] + : +- Filter ((outer(t1a#x) = t2a#x) OR (outer(t1b#x) > t2b#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t2i IN (SELECT t3i + FROM t3 + WHERE t2c = t3c)) +-- !query analysis +Project [t1a#x, t1b#x] ++- Filter t1c#x IN (list#x []) + : +- Project [cast(t2b#x as int) AS t2b#x] + : +- Project [t2b#x] + : +- Filter t2i#x IN (list#x [t2c#x]) + : : +- Project [t3i#x] + : : +- Filter (outer(t2c#x) = t3c#x) + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t2a IN (SELECT t3a + FROM t3 + WHERE t2c = t3c + AND t2b IS NOT NULL)) +-- !query analysis +Project [t1a#x, t1b#x] ++- Filter t1c#x IN (list#x []) + : +- Project [cast(t2b#x as int) AS t2b#x] + : +- Project [t2b#x] + : +- Filter t2a#x IN (list#x [t2c#x && t2b#x]) + : : +- Project [t3a#x] + : : +- Filter ((outer(t2c#x) = t3c#x) AND isnotnull(outer(t2b#x))) + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT DISTINCT( t1a ), + t1b, + t1h +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2) +-- !query analysis +Distinct ++- Project [t1a#x, t1b#x, t1h#x] + +- Filter NOT t1a#x IN (list#x []) + : +- Project [t2a#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +create temporary view a as select * from values + (1, 1), (2, 1), (null, 1), (1, 3), (null, 3), (1, null), (null, 2) + as a(a1, a2) +-- !query analysis +CreateViewCommand `a`, select * from values + (1, 1), (2, 1), (null, 1), (1, 3), (null, 3), (1, null), (null, 2) + as a(a1, a2), false, false, LocalTempView, true + +- Project [a1#x, a2#x] + +- SubqueryAlias a + +- LocalRelation [a1#x, a2#x] + + +-- !query +create temporary view b as select * from values + (1, 1, 2), (null, 3, 2), (1, null, 2), (1, 2, null) + as b(b1, b2, b3) +-- !query analysis +CreateViewCommand `b`, select * from values + (1, 1, 2), (null, 3, 2), (1, null, 2), (1, 2, null) + as b(b1, b2, b3), false, false, LocalTempView, true + +- Project [b1#x, b2#x, b3#x] + +- SubqueryAlias b + +- LocalRelation [b1#x, b2#x, b3#x] + + +-- !query +SELECT a1, a2 +FROM a +WHERE a1 NOT IN (SELECT b.b1 + FROM b + WHERE a.a2 = b.b2) +-- !query analysis +Project [a1#x, a2#x] ++- Filter NOT a1#x IN (list#x [a2#x]) + : +- Project [b1#x] + : +- Filter (outer(a2#x) = b2#x) + : +- SubqueryAlias b + : +- View (`b`, [b1#x,b2#x,b3#x]) + : +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x] + : +- Project [b1#x, b2#x, b3#x] + : +- SubqueryAlias b + : +- LocalRelation [b1#x, b2#x, b3#x] + +- SubqueryAlias a + +- View (`a`, [a1#x,a2#x]) + +- Project [cast(a1#x as int) AS a1#x, cast(a2#x as int) AS a2#x] + +- Project [a1#x, a2#x] + +- SubqueryAlias a + +- LocalRelation [a1#x, a2#x] + + +-- !query +SELECT a1, a2 +FROM a +WHERE a1 NOT IN (SELECT b.b1 + FROM b + WHERE a.a2 = b.b2 + AND b.b3 > 1) +-- !query analysis +Project [a1#x, a2#x] ++- Filter NOT a1#x IN (list#x [a2#x]) + : +- Project [b1#x] + : +- Filter ((outer(a2#x) = b2#x) AND (b3#x > 1)) + : +- SubqueryAlias b + : +- View (`b`, [b1#x,b2#x,b3#x]) + : +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x] + : +- Project [b1#x, b2#x, b3#x] + : +- SubqueryAlias b + : +- LocalRelation [b1#x, b2#x, b3#x] + +- SubqueryAlias a + +- View (`a`, [a1#x,a2#x]) + +- Project [cast(a1#x as int) AS a1#x, cast(a2#x as int) AS a2#x] + +- Project [a1#x, a2#x] + +- SubqueryAlias a + +- LocalRelation [a1#x, a2#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out new file mode 100644 index 0000000000000..efecd78a82fa8 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out @@ -0,0 +1,205 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c) +-- !query analysis +CreateViewCommand `t1`, SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x] + + +-- !query +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES + (1, 0, 1) +AS t2(t2a, t2b, t2c) +-- !query analysis +CreateViewCommand `t2`, SELECT * FROM VALUES + (1, 0, 1) +AS t2(t2a, t2b, t2c), false, false, LocalTempView, true + +- Project [t2a#x, t2b#x, t2c#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x] + + +-- !query +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES + (3, 1, 2) +AS t3(t3a, t3b, t3c) +-- !query analysis +CreateViewCommand `t3`, SELECT * FROM VALUES + (3, 1, 2) +AS t3(t3a, t3b, t3c), false, false, LocalTempView, true + +- Project [t3a#x, t3b#x, t3c#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x] + + +-- !query +SELECT t1a, t2b +FROM t1, t2 +WHERE t1b = t2c +AND t2b = (SELECT max(avg) + FROM (SELECT t2b, avg(t2b) avg + FROM t2 + WHERE t2a = t1.t1b + ) + ) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 100, + "stopIndex" : 203, + "fragment" : "SELECT t2b, avg(t2b) avg\n FROM t2\n WHERE t2a = t1.t1b" + } ] +} + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT min(t2a) + FROM t2 + GROUP BY t2c + HAVING t2c IN (SELECT max(t3c) + FROM t3 + GROUP BY t3b + HAVING t3b > t2b )) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2432", + "messageParameters" : { + "msg" : "Resolved attribute(s) t2b#x missing from min(t2a)#x,t2c#x in operator !Filter t2c#x IN (list#x [t2b#x])." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 35, + "stopIndex" : 298, + "fragment" : "SELECT min(t2a)\n FROM t2\n GROUP BY t2c\n HAVING t2c IN (SELECT max(t3c)\n FROM t3\n GROUP BY t3b\n HAVING t3b > t2b )" + } ] +} + + +-- !query +SELECT t1a +FROM t1 +GROUP BY 1 +HAVING EXISTS (SELECT t2a + FROM t2 + GROUP BY 1 + HAVING t2a < min(t1a + t2a)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "min((outer(t1.t1a) + t2.t2a))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 139, + "stopIndex" : 152, + "fragment" : "min(t1a + t2a)" + } ] +} + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE EXISTS (SELECT 1 + FROM t3 + GROUP BY 1 + HAVING min(t2a + t3a) > 1)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "min((outer(t2.t2a) + t3.t3a))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 233, + "stopIndex" : 246, + "fragment" : "min(t2a + t3a)" + } ] +} + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE EXISTS (SELECT min(t2a) + FROM t3)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"min(t2a) AS `min(outer(t2.t2a))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 105, + "stopIndex" : 160, + "fragment" : "SELECT min(t2a) \n FROM t3" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW t1_copy AS SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c) +-- !query analysis +CreateViewCommand `t1_copy`, SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x] + + +-- !query +SELECT t1.t1a +FROM t1 +JOIN t1_copy +ON EXISTS (SELECT 1 FROM t2 WHERE t2a > t1a) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1a`", + "proposal" : "`t2`.`t2a`, `t2`.`t2b`, `t2`.`t2c`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 84, + "stopIndex" : 86, + "fragment" : "t1a" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/subq-input-typecheck.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/subq-input-typecheck.sql.out new file mode 100644 index 0000000000000..864c589184e5d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/subq-input-typecheck.sql.out @@ -0,0 +1,200 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c) +-- !query analysis +CreateViewCommand `t1`, SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x] + + +-- !query +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES + (1, 0, 1) +AS t2(t2a, t2b, t2c) +-- !query analysis +CreateViewCommand `t2`, SELECT * FROM VALUES + (1, 0, 1) +AS t2(t2a, t2b, t2c), false, false, LocalTempView, true + +- Project [t2a#x, t2b#x, t2c#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x] + + +-- !query +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES + (3, 1, 2) +AS t3(t3a, t3b, t3c) +-- !query analysis +CreateViewCommand `t3`, SELECT * FROM VALUES + (3, 1, 2) +AS t3(t3a, t3b, t3c), false, false, LocalTempView, true + +- Project [t3a#x, t3b#x, t3c#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x] + + +-- !query +CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES + (CAST(1 AS DOUBLE), CAST(2 AS STRING), CAST(3 AS STRING)) +AS t1(t4a, t4b, t4c) +-- !query analysis +CreateViewCommand `t4`, SELECT * FROM VALUES + (CAST(1 AS DOUBLE), CAST(2 AS STRING), CAST(3 AS STRING)) +AS t1(t4a, t4b, t4c), false, false, LocalTempView, true + +- Project [t4a#x, t4b#x, t4c#x] + +- SubqueryAlias t1 + +- LocalRelation [t4a#x, t4b#x, t4c#x] + + +-- !query +CREATE TEMPORARY VIEW t5 AS SELECT * FROM VALUES + (CAST('2011-01-01 01:01:01' AS TIMESTAMP), CAST(2 AS STRING), CAST(3 AS BIGINT)) +AS t1(t5a, t5b, t5c) +-- !query analysis +CreateViewCommand `t5`, SELECT * FROM VALUES + (CAST('2011-01-01 01:01:01' AS TIMESTAMP), CAST(2 AS STRING), CAST(3 AS BIGINT)) +AS t1(t5a, t5b, t5c), false, false, LocalTempView, true + +- Project [t5a#x, t5b#x, t5c#xL] + +- SubqueryAlias t1 + +- LocalRelation [t5a#x, t5b#x, t5c#xL] + + +-- !query +SELECT + ( SELECT max(t2b), min(t2b) + FROM t2 + WHERE t2.t2b = t1.t1b + GROUP BY t2.t2b + ) +FROM t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION.SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", + "sqlState" : "42823", + "messageParameters" : { + "number" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 11, + "stopIndex" : 101, + "fragment" : "( SELECT max(t2b), min(t2b) \n FROM t2 \n WHERE t2.t2b = t1.t1b\n GROUP BY t2.t2b\n )" + } ] +} + + +-- !query +SELECT + ( SELECT max(t2b), min(t2b) + FROM t2 + WHERE t2.t2b > 0 + GROUP BY t2.t2b + ) +FROM t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION.SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", + "sqlState" : "42823", + "messageParameters" : { + "number" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 11, + "stopIndex" : 96, + "fragment" : "( SELECT max(t2b), min(t2b) \n FROM t2 \n WHERE t2.t2b > 0\n GROUP BY t2.t2b\n )" + } ] +} + + +-- !query +SELECT * FROM t1 +WHERE +t1a IN (SELECT t2a, t2b + FROM t2 + WHERE t1a = t2a) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_LENGTH_MISMATCH", + "sqlState" : "42K09", + "messageParameters" : { + "leftColumns" : "\"t1a\"", + "leftLength" : "1", + "rightColumns" : "\"t2a\", \"t2b\"", + "rightLength" : "2", + "sqlExpr" : "\"(t1a IN (listquery(t1a)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 88, + "fragment" : "IN (SELECT t2a, t2b \n FROM t2\n WHERE t1a = t2a)" + } ] +} + + +-- !query +SELECT * FROM T1 +WHERE +(t1a, t1b) IN (SELECT t2a + FROM t2 + WHERE t1a = t2a) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_LENGTH_MISMATCH", + "sqlState" : "42K09", + "messageParameters" : { + "leftColumns" : "\"t1a\", \"t1b\"", + "leftLength" : "2", + "rightColumns" : "\"t2a\"", + "rightLength" : "1", + "sqlExpr" : "\"(named_struct('t1a', t1a, 't1b', t1b) IN (listquery(t1a)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 104, + "fragment" : "IN (SELECT t2a\n FROM t2\n WHERE t1a = t2a)" + } ] +} + + +-- !query +SELECT * FROM t4 +WHERE +(t4a, t4b, t4c) IN (SELECT t5a, + t5b, + t5c + FROM t5) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_DATA_TYPE_MISMATCH", + "sqlState" : "42K09", + "messageParameters" : { + "leftType" : "\"DOUBLE\", \"STRING\", \"STRING\"", + "mismatchedColumns" : "(t4.t4a:double, t5.t5a:timestamp), (t4.t4c:string, t5.t5c:bigint)", + "rightType" : "\"TIMESTAMP\", \"STRING\", \"BIGINT\"", + "sqlExpr" : "\"(named_struct('t4a', t4a, 't4b', t4b, 't4c', t4c) IN (listquery()))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 40, + "stopIndex" : 146, + "fragment" : "IN (SELECT t5a,\n t5b,\n t5c\n FROM t5)" + } ] +} 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 new file mode 100644 index 0000000000000..789c864dea815 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out @@ -0,0 +1,1243 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW p AS VALUES (1, 1) AS T(pk, pv) +-- !query analysis +CreateViewCommand `p`, VALUES (1, 1) AS T(pk, pv), false, true, LocalTempView, true + +- SubqueryAlias T + +- LocalRelation [pk#x, pv#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW c AS VALUES (1, 1) AS T(ck, cv) +-- !query analysis +CreateViewCommand `c`, VALUES (1, 1) AS T(ck, cv), false, true, LocalTempView, true + +- SubqueryAlias T + +- LocalRelation [ck#x, cv#x] + + +-- !query +SELECT pk, cv +FROM p, c +WHERE p.pk = c.ck +AND c.cv = (SELECT avg(c1.cv) + FROM c c1 + WHERE c1.ck = p.pk) +-- !query analysis +Project [pk#x, cv#x] ++- Filter ((pk#x = ck#x) AND (cast(cv#x as double) = scalar-subquery#x [pk#x])) + : +- Aggregate [avg(cv#x) AS avg(cv)#x] + : +- Filter (ck#x = outer(pk#x)) + : +- SubqueryAlias c1 + : +- SubqueryAlias c + : +- View (`c`, [ck#x,cv#x]) + : +- Project [cast(ck#x as int) AS ck#x, cast(cv#x as int) AS cv#x] + : +- SubqueryAlias T + : +- LocalRelation [ck#x, cv#x] + +- Join Inner + :- SubqueryAlias p + : +- View (`p`, [pk#x,pv#x]) + : +- Project [cast(pk#x as int) AS pk#x, cast(pv#x as int) AS pv#x] + : +- SubqueryAlias T + : +- LocalRelation [pk#x, pv#x] + +- SubqueryAlias c + +- View (`c`, [ck#x,cv#x]) + +- Project [cast(ck#x as int) AS ck#x, cast(cv#x as int) AS cv#x] + +- SubqueryAlias T + +- LocalRelation [ck#x, cv#x] + + +-- !query +SELECT pk, cv +FROM p, c +WHERE p.pk = c.ck +AND c.cv = (SELECT max(avg) + FROM (SELECT c1.cv, avg(c1.cv) avg + FROM c c1 + WHERE c1.ck = p.pk + GROUP BY c1.cv)) +-- !query analysis +Project [pk#x, cv#x] ++- Filter ((pk#x = ck#x) AND (cast(cv#x as double) = scalar-subquery#x [pk#x])) + : +- Aggregate [max(avg#x) AS max(avg)#x] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Aggregate [cv#x], [cv#x, avg(cv#x) AS avg#x] + : +- Filter (ck#x = outer(pk#x)) + : +- SubqueryAlias c1 + : +- SubqueryAlias c + : +- View (`c`, [ck#x,cv#x]) + : +- Project [cast(ck#x as int) AS ck#x, cast(cv#x as int) AS cv#x] + : +- SubqueryAlias T + : +- LocalRelation [ck#x, cv#x] + +- Join Inner + :- SubqueryAlias p + : +- View (`p`, [pk#x,pv#x]) + : +- Project [cast(pk#x as int) AS pk#x, cast(pv#x as int) AS pv#x] + : +- SubqueryAlias T + : +- LocalRelation [pk#x, pv#x] + +- SubqueryAlias c + +- View (`c`, [ck#x,cv#x]) + +- Project [cast(ck#x as int) AS ck#x, cast(cv#x as int) AS cv#x] + +- SubqueryAlias T + +- LocalRelation [ck#x, cv#x] + + +-- !query +create temporary view t1 as select * from values + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), + ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ('val1d', null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ('val1d', null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query analysis +CreateViewCommand `t1`, select * from values + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), + ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ('val1d', null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ('val1d', null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +create temporary view t2 as select * from values + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query analysis +CreateViewCommand `t2`, select * from values + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i), false, false, LocalTempView, true + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +create temporary view t3 as select * from values + ('val3a', 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ('val3a', 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ('val3c', 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ('val3c', 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ('val1b', null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ('val3b', 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val3b', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query analysis +CreateViewCommand `t3`, select * from values + ('val3a', 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ('val3a', 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ('val3c', 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ('val3c', 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ('val1b', null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ('val3b', 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val3b', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i), false, false, LocalTempView, true + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT t1a, t1b +FROM t1 +WHERE t1c = (SELECT max(t2c) + FROM t2) +-- !query analysis +Project [t1a#x, t1b#x] ++- Filter (t1c#x = scalar-subquery#x []) + : +- Aggregate [max(t2c#x) AS max(t2c)#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, t1d, t1f +FROM t1 +WHERE t1c = (SELECT max(t2c) + FROM t2) +AND t1b > (SELECT min(t3b) + FROM t3) +-- !query analysis +Project [t1a#x, t1d#xL, t1f#x] ++- Filter ((t1c#x = scalar-subquery#x []) AND (t1b#x > scalar-subquery#x [])) + : :- Aggregate [max(t2c#x) AS max(t2c)#x] + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Aggregate [min(t3b#x) AS min(t3b)#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, t1h +FROM t1 +WHERE t1c = (SELECT max(t2c) + FROM t2) +OR t1b = (SELECT min(t3b) + FROM t3 + WHERE t3b > 10) +-- !query analysis +Project [t1a#x, t1h#x] ++- Filter ((t1c#x = scalar-subquery#x []) OR (t1b#x = scalar-subquery#x [])) + : :- Aggregate [max(t2c#x) AS max(t2c)#x] + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Aggregate [min(t3b#x) AS min(t3b)#x] + : +- Filter (cast(t3b#x as int) > 10) + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, t1b, t2d +FROM t1 LEFT JOIN t2 + ON t1a = t2a +WHERE t1b = (SELECT min(t3b) + FROM t3) +-- !query analysis +Project [t1a#x, t1b#x, t2d#xL] ++- Filter (t1b#x = scalar-subquery#x []) + : +- Aggregate [min(t3b#x) AS min(t3b)#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- Join LeftOuter, (t1a#x = t2a#x) + :- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t2 + +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +SELECT t1a, t1b, t1g +FROM t1 +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 []) + : +- 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]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, t1h +FROM t1 +WHERE date(t1h) = (SELECT min(t2i) + FROM t2) +-- !query analysis +Project [t1a#x, t1h#x] ++- Filter (cast(t1h#x as date) = scalar-subquery#x []) + : +- Aggregate [min(t2i#x) AS min(t2i)#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t2d, t1a +FROM t1, t2 +WHERE t1b = t2b +AND t2c + 1 = (SELECT max(t2c) + 1 + FROM t2, t1 + WHERE t2b = t1b) +-- !query analysis +Project [t2d#xL, t1a#x] ++- Filter ((t1b#x = t2b#x) AND ((t2c#x + 1) = scalar-subquery#x [])) + : +- Aggregate [(max(t2c#x) + 1) AS (max(t2c) + 1)#x] + : +- Filter (t2b#x = t1b#x) + : +- Join Inner + : :- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Join Inner + :- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t2 + +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +SELECT DISTINCT t2a, max_t1g +FROM t2, (SELECT max(t1g) max_t1g, t1a + FROM t1 + GROUP BY t1a) t1 +WHERE t2a = t1a +AND max_t1g = (SELECT max(t1g) + FROM t1) +-- !query analysis +Distinct ++- Project [t2a#x, max_t1g#x] + +- Filter ((t2a#x = t1a#x) AND (max_t1g#x = scalar-subquery#x [])) + : +- Aggregate [max(t1g#x) AS max(t1g)#x] + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Join Inner + :- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- Aggregate [t1a#x], [max(t1g#x) AS max_t1g#x, t1a#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t3b, t3c +FROM t3 +WHERE (SELECT max(t3c) + FROM t3 + WHERE t3b > 10) >= + (SELECT min(t3b) + FROM t3 + WHERE t3c > 0) +AND (t3b is null or t3c is null) +-- !query analysis +Project [t3b#x, t3c#x] ++- Filter ((scalar-subquery#x [] >= cast(scalar-subquery#x [] as int)) AND (isnull(t3b#x) OR isnull(t3c#x))) + : :- Aggregate [max(t3c#x) AS max(t3c)#x] + : : +- Filter (cast(t3b#x as int) > 10) + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- Aggregate [min(t3b#x) AS min(t3b)#x] + : +- Filter (t3c#x > 0) + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as decimal(4,0)) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a < (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query analysis +Project [t1a#x] ++- Filter (t1a#x < scalar-subquery#x [t1c#x]) + : +- Aggregate [t2c#x], [max(t2a#x) AS max(t2a)#x] + : +- Filter (t2c#x = outer(t1c#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, t1c +FROM t1 +WHERE (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) IS NULL +-- !query analysis +Project [t1a#x, t1c#x] ++- Filter isnull(scalar-subquery#x [t1c#x]) + : +- Aggregate [t2c#x], [max(t2a#x) AS max(t2a)#x] + : +- Filter (t2c#x = outer(t1c#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a = (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c + HAVING count(*) >= 0) +OR t1i > '2014-12-31' +-- !query analysis +Project [t1a#x] ++- Filter ((t1a#x = scalar-subquery#x [t1c#x]) OR (t1i#x > cast(2014-12-31 as date))) + : +- Project [max(t2a)#x] + : +- Filter (count(1)#xL >= cast(0 as bigint)) + : +- Aggregate [t2c#x], [max(t2a#x) AS max(t2a)#x, count(1) AS count(1)#xL] + : +- Filter (t2c#x = outer(t1c#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a = (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c + HAVING count(*) >= 1) +OR t1i > '2014-12-31' +-- !query analysis +Project [t1a#x] ++- Filter ((t1a#x = scalar-subquery#x [t1c#x]) OR (t1i#x > cast(2014-12-31 as date))) + : +- Project [max(t2a)#x] + : +- Filter (count(1)#xL >= cast(1 as bigint)) + : +- Aggregate [t2c#x], [max(t2a#x) AS max(t2a)#x, count(1) AS count(1)#xL] + : +- Filter (t2c#x = outer(t1c#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT count(t1a) +FROM t1 RIGHT JOIN t2 +ON t1d = t2d +WHERE t1a < (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query analysis +Aggregate [count(t1a#x) AS count(t1a)#xL] ++- Filter (t1a#x < scalar-subquery#x [t1c#x]) + : +- Aggregate [t2c#x], [max(t2a#x) AS max(t2a)#x] + : +- Filter (t2c#x = outer(t1c#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Join RightOuter, (t1d#xL = t2d#xL) + :- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t2 + +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +SELECT t1a +FROM t1 +WHERE t1b <= (SELECT max(t2b) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +AND t1b >= (SELECT min(t2b) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query analysis +Project [t1a#x] ++- Filter ((t1b#x <= scalar-subquery#x [t1c#x]) AND (t1b#x >= scalar-subquery#x [t1c#x])) + : :- Aggregate [t2c#x], [max(t2b#x) AS max(t2b)#x] + : : +- Filter (t2c#x = outer(t1c#x)) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- Aggregate [t2c#x], [min(t2b#x) AS min(t2b)#x] + : +- Filter (t2c#x = outer(t1c#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +INTERSECT +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query analysis +Intersect false +:- Project [t1a#x] +: +- Filter (t1a#x <= scalar-subquery#x [t1c#x]) +: : +- Aggregate [t2c#x], [max(t2a#x) AS max(t2a)#x] +: : +- Filter (t2c#x = outer(t1c#x)) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Project [t1a#x] + +- Filter (t1a#x >= scalar-subquery#x [t1c#x]) + : +- Aggregate [t2c#x], [min(t2a#x) AS min(t2a)#x] + : +- Filter (t2c#x = outer(t1c#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +UNION ALL +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query analysis +Union false, false +:- Project [t1a#x] +: +- Filter (t1a#x <= scalar-subquery#x [t1c#x]) +: : +- Aggregate [t2c#x], [max(t2a#x) AS max(t2a)#x] +: : +- Filter (t2c#x = outer(t1c#x)) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Project [t1a#x] + +- Filter (t1a#x >= scalar-subquery#x [t1c#x]) + : +- Aggregate [t2c#x], [min(t2a#x) AS min(t2a)#x] + : +- Filter (t2c#x = outer(t1c#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +UNION DISTINCT +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query analysis +Distinct ++- Union false, false + :- Project [t1a#x] + : +- Filter (t1a#x <= scalar-subquery#x [t1c#x]) + : : +- Aggregate [t2c#x], [max(t2a#x) AS max(t2a)#x] + : : +- Filter (t2c#x = outer(t1c#x)) + : : +- SubqueryAlias t2 + : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Project [t1a#x] + +- Filter (t1a#x >= scalar-subquery#x [t1c#x]) + : +- Aggregate [t2c#x], [min(t2a#x) AS min(t2a)#x] + : +- Filter (t2c#x = outer(t1c#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +MINUS +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query analysis +Except false +:- Project [t1a#x] +: +- Filter (t1a#x <= scalar-subquery#x [t1c#x]) +: : +- Aggregate [t2c#x], [max(t2a#x) AS max(t2a)#x] +: : +- Filter (t2c#x = outer(t1c#x)) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Project [t1a#x] + +- Filter (t1a#x >= scalar-subquery#x [t1c#x]) + : +- Aggregate [t2c#x], [min(t2a#x) AS min(t2a)#x] + : +- Filter (t2c#x = outer(t1c#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a +FROM t1 +GROUP BY t1a, t1c +HAVING max(t1b) <= (SELECT max(t2b) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query analysis +Project [t1a#x] ++- Filter (max(t1b#x)#x <= scalar-subquery#x [t1c#x]) + : +- Aggregate [t2c#x], [max(t2b#x) AS max(t2b)#x] + : +- Filter (t2c#x = outer(t1c#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as decimal(4,0)) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Aggregate [t1a#x, t1c#x], [t1a#x, max(t1b#x) AS max(t1b#x)#x, t1c#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as decimal(4,0)) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +CREATE OR REPLACE TEMP VIEW t0(t0a, t0b) AS VALUES (1, 1), (2, 0) +-- !query analysis +CreateViewCommand `t0`, [(t0a,None), (t0b,None)], VALUES (1, 1), (2, 0), false, true, LocalTempView, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE OR REPLACE TEMP VIEW t1(t1a, t1b, t1c) AS VALUES (1, 1, 3) +-- !query analysis +CreateViewCommand `t1`, [(t1a,None), (t1b,None), (t1c,None)], VALUES (1, 1, 3), false, true, LocalTempView, true + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +CREATE OR REPLACE TEMP VIEW t2(t2a, t2b, t2c) AS VALUES (1, 1, 5), (2, 2, 7) +-- !query analysis +CreateViewCommand `t2`, [(t2a,None), (t2b,None), (t2c,None)], VALUES (1, 1, 5), (2, 2, 7), false, true, LocalTempView, true + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +-- !query analysis +Project [t0a#x, t0b#x] ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0b#x]) + : +- Aggregate [sum(c#x) AS sum(c)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Union false, false + : :- Project [t1c#x AS c#x] + : : +- Filter (t1a#x = outer(t0a#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) + : : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] + : : +- LocalRelation [col1#x, col2#x, col3#x] + : +- Project [t2c#x AS c#x] + : +- Filter (t2b#x = outer(t0b#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x]) + : +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +-- !query analysis +Project [t0a#x, t0b#x] ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0a#x]) + : +- Aggregate [sum(c#x) AS sum(c)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Union false, false + : :- Project [t1c#x AS c#x] + : : +- Filter (t1a#x = outer(t0a#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) + : : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] + : : +- LocalRelation [col1#x, col2#x, col3#x] + : +- Project [t2c#x AS c#x] + : +- Filter (t2a#x = outer(t0a#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x]) + : +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +-- !query analysis +Project [t0a#x, t0b#x] ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0b#x]) + : +- Aggregate [sum(c#x) AS sum(c)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Union false, false + : :- Project [t1c#x AS c#x] + : : +- Filter (t1a#x > outer(t0a#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) + : : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] + : : +- LocalRelation [col1#x, col2#x, col3#x] + : +- Project [t2c#x AS c#x] + : +- Filter (t2b#x <= outer(t0b#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x]) + : +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +-- !query analysis +Project [t0a#x, t0b#x] ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0b#x]) + : +- Aggregate [sum(t1c#x) AS sum(t1c)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Union false, false + : :- Project [t1c#x] + : : +- Filter (t1a#x = outer(t0a#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) + : : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] + : : +- LocalRelation [col1#x, col2#x, col3#x] + : +- Project [t2c#x] + : +- Filter (t2b#x = outer(t0b#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x]) + : +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +-- !query analysis +Project [t0a#x, t0b#x] ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0b#x]) + : +- Aggregate [sum(t1c#x) AS sum(t1c)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Distinct + : +- Union false, false + : :- Project [t1c#x] + : : +- Filter (t1a#x = outer(t0a#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) + : : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] + : : +- LocalRelation [col1#x, col2#x, col3#x] + : +- Project [t2c#x] + : +- Filter (t2b#x = outer(t0b#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x]) + : +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +-- !query analysis +Project [t0a#x, t0b#x] ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x]) + : +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Union false, false + : :- Project [t1c#x AS t1a#x, t1a#x AS t1b#x, outer(t0a#x) AS t1c#x] + : : +- Filter (t1a#x = outer(t0a#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) + : : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] + : : +- LocalRelation [col1#x, col2#x, col3#x] + : +- Project [outer(t0a#x) AS t2b#x, t2c#x AS t1a#x, outer(t0b#x) AS t2c#x] + : +- Filter (t2b#x = outer(t0b#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x]) + : +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +-- !query analysis +Project [t0a#x, t0b#x] ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0b#x]) + : +- Aggregate [count(t1c#x) AS count(t1c)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Distinct + : +- Union false, false + : :- Project [t1c#x] + : : +- Filter (t1a#x = outer(t0a#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) + : : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] + : : +- LocalRelation [col1#x, col2#x, col3#x] + : +- Project [t2c#x] + : +- Filter (t2b#x = outer(t0b#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x]) + : +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + UNION ALL + SELECT t2a - t0a as d + FROM t2) +) +-- !query analysis +Project [t0a#x, t0b#x] ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0a#x]) + : +- Aggregate [sum(d#x) AS sum(d)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Union false, false + : :- Project [(t1a#x - outer(t0a#x)) AS d#x] + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) + : : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] + : : +- LocalRelation [col1#x, col2#x, col3#x] + : +- Project [(t2a#x - outer(t0a#x)) AS d#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x]) + : +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + UNION ALL + SELECT sum(t2a) + t0a as d + FROM t2) +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(t0a) AS d\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 53, + "stopIndex" : 84, + "fragment" : "SELECT sum(t0a) as d\n FROM t1" + } ] +} + + +-- !query +SELECT t0a, t0b FROM t0 +GROUP BY t0a, t0b +HAVING t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +-- !query analysis +Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0b#x]) +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Union false, false +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x > outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2b#x <= outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- Aggregate [t0a#x, t0b#x], [t0a#x, t0b#x] + +- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out new file mode 100644 index 0000000000000..b4de465903ca2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -0,0 +1,974 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), + ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ('val1d', null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ('val1d', null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query analysis +CreateViewCommand `t1`, select * from values + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), + ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ('val1d', null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ('val1d', null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i), false, false, LocalTempView, true + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +create temporary view t2 as select * from values + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query analysis +CreateViewCommand `t2`, select * from values + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i), false, false, LocalTempView, true + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +create temporary view t3 as select * from values + ('val3a', 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ('val3a', 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ('val3c', 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ('val3c', 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ('val1b', null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ('val3b', 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val3b', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query analysis +CreateViewCommand `t3`, select * from values + ('val3a', 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ('val3a', 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ('val3c', 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ('val3c', 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ('val1b', null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ('val3b', 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val3b', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i), false, false, LocalTempView, true + +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + + +-- !query +SELECT (SELECT min(t3d) FROM t3) min_t3d, + (SELECT max(t2h) FROM t2) max_t2h +FROM t1 +WHERE t1a = 'val1c' +-- !query analysis +Project [scalar-subquery#x [] AS min_t3d#xL, scalar-subquery#x [] AS max_t2h#x] +: :- Aggregate [min(t3d#xL) AS min(t3d)#xL] +: : +- SubqueryAlias t3 +: : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) +: : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] +: : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: : +- SubqueryAlias t3 +: : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: +- Aggregate [max(t2h#x) AS max(t2h)#x] +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- SubqueryAlias t2 +: +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] ++- Filter (t1a#x = val1c) + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, count(*) +FROM t1 +WHERE t1c IN (SELECT (SELECT min(t3c) FROM t3) + FROM t2 + GROUP BY t2g + HAVING count(*) > 1) +GROUP BY t1a +-- !query analysis +Aggregate [t1a#x], [t1a#x, count(1) AS count(1)#xL] ++- Filter t1c#x IN (list#x []) + : +- Project [scalarsubquery()#x] + : +- Filter (count(1)#xL > cast(1 as bigint)) + : +- Aggregate [t2g#x], [scalar-subquery#x [] AS scalarsubquery()#x, count(1) AS count(1)#xL] + : : +- Aggregate [min(t3c#x) AS min(t3c)#x] + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT (SELECT min(t3d) FROM t3) min_t3d, + null +FROM t1 +WHERE t1a = 'val1c' +UNION +SELECT null, + (SELECT max(t2h) FROM t2) max_t2h +FROM t1 +WHERE t1a = 'val1c' +-- !query analysis +Distinct ++- Union false, false + :- Project [min_t3d#xL, cast(NULL#x as timestamp) AS NULL#x] + : +- Project [scalar-subquery#x [] AS min_t3d#xL, null AS NULL#x] + : : +- Aggregate [min(t3d#xL) AS min(t3d)#xL] + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- Filter (t1a#x = val1c) + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- Project [cast(NULL#x as bigint) AS NULL#xL, max_t2h#x] + +- Project [null AS NULL#x, scalar-subquery#x [] AS max_t2h#x] + : +- Aggregate [max(t2h#x) AS max(t2h)#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Filter (t1a#x = val1c) + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT (SELECT min(t3c) FROM t3) min_t3d +FROM t1 +WHERE t1a = 'val1a' +INTERSECT +SELECT (SELECT min(t2c) FROM t2) min_t2d +FROM t1 +WHERE t1a = 'val1d' +-- !query analysis +Intersect false +:- Project [scalar-subquery#x [] AS min_t3d#x] +: : +- Aggregate [min(t3c#x) AS min(t3c)#x] +: : +- SubqueryAlias t3 +: : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) +: : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] +: : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: : +- SubqueryAlias t3 +: : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: +- Filter (t1a#x = val1a) +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Project [scalar-subquery#x [] AS min_t2d#x] + : +- Aggregate [min(t2c#x) AS min(t2c)#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : +- SubqueryAlias t2 + : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- Filter (t1a#x = val1d) + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT q1.t1a, q2.t2a, q1.min_t3d, q2.avg_t3d +FROM (SELECT t1a, (SELECT min(t3d) FROM t3) min_t3d + FROM t1 + WHERE t1a IN ('val1e', 'val1c')) q1 + FULL OUTER JOIN + (SELECT t2a, (SELECT avg(t3d) FROM t3) avg_t3d + FROM t2 + WHERE t2a IN ('val1c', 'val2a')) q2 +ON q1.t1a = q2.t2a +AND q1.min_t3d < q2.avg_t3d +-- !query analysis +Project [t1a#x, t2a#x, min_t3d#xL, avg_t3d#x] ++- Join FullOuter, ((t1a#x = t2a#x) AND (cast(min_t3d#xL as double) < avg_t3d#x)) + :- SubqueryAlias q1 + : +- Project [t1a#x, scalar-subquery#x [] AS min_t3d#xL] + : : +- Aggregate [min(t3d#xL) AS min(t3d)#xL] + : : +- SubqueryAlias t3 + : : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : : +- SubqueryAlias t3 + : : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- Filter t1a#x IN (val1e,val1c) + : +- SubqueryAlias t1 + : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t1 + : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias q2 + +- Project [t2a#x, scalar-subquery#x [] AS avg_t3d#x] + : +- Aggregate [avg(t3d#xL) AS avg(t3d)#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- Filter t2a#x IN (val1c,val2a) + +- SubqueryAlias t2 + +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + +- SubqueryAlias t2 + +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + + +-- !query +SELECT (SELECT min(t3d) FROM t3 WHERE t3.t3a = t1.t1a) min_t3d, + (SELECT max(t2h) FROM t2 WHERE t2.t2a = t1.t1a) max_t2h +FROM t1 +WHERE t1a = 'val1b' +-- !query analysis +Project [scalar-subquery#x [t1a#x] AS min_t3d#xL, scalar-subquery#x [t1a#x] AS max_t2h#x] +: :- Aggregate [min(t3d#xL) AS min(t3d)#xL] +: : +- Filter (t3a#x = outer(t1a#x)) +: : +- SubqueryAlias t3 +: : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) +: : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] +: : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: : +- SubqueryAlias t3 +: : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: +- Aggregate [max(t2h#x) AS max(t2h)#x] +: +- Filter (t2a#x = outer(t1a#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- SubqueryAlias t2 +: +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] ++- Filter (t1a#x = val1b) + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT (SELECT min(t3d) FROM t3 WHERE t3a = t1a) min_t3d +FROM t1 +WHERE t1a = 'val1b' +MINUS +SELECT (SELECT min(t3d) FROM t3) abs_min_t3d +FROM t1 +WHERE t1a = 'val1b' +-- !query analysis +Except false +:- Project [scalar-subquery#x [t1a#x] AS min_t3d#xL] +: : +- Aggregate [min(t3d#xL) AS min(t3d)#xL] +: : +- Filter (t3a#x = outer(t1a#x)) +: : +- SubqueryAlias t3 +: : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) +: : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] +: : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: : +- SubqueryAlias t3 +: : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] +: +- Filter (t1a#x = val1b) +: +- SubqueryAlias t1 +: +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +: +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] +: +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] +: +- SubqueryAlias t1 +: +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] ++- Project [scalar-subquery#x [] AS abs_min_t3d#xL] + : +- Aggregate [min(t3d#xL) AS min(t3d)#xL] + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- Filter (t1a#x = val1b) + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, t1b +FROM t1 +WHERE NOT EXISTS (SELECT (SELECT max(t2b) + FROM t2 LEFT JOIN t1 + ON t2a = t1a + WHERE t2c = t3c) dummy + FROM t3 + WHERE t3b < (SELECT max(t2b) + FROM t2 LEFT JOIN t1 + ON t2a = t1a + WHERE t2c = t3c) + AND t3a = t1a) +-- !query analysis +Project [t1a#x, t1b#x] ++- Filter NOT exists#x [t1a#x] + : +- Project [scalar-subquery#x [t3c#x] AS dummy#x] + : : +- Aggregate [max(t2b#x) AS max(t2b)#x] + : : +- Filter (t2c#x = outer(t3c#x)) + : : +- Join LeftOuter, (t2a#x = t1a#x) + : : :- SubqueryAlias t2 + : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- SubqueryAlias t2 + : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- Filter ((t3b#x < scalar-subquery#x [t3c#x]) AND (t3a#x = outer(t1a#x))) + : : +- Aggregate [max(t2b#x) AS max(t2b)#x] + : : +- Filter (t2c#x = outer(t3c#x)) + : : +- Join LeftOuter, (t2a#x = t1a#x) + : : :- SubqueryAlias t2 + : : : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) + : : : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] + : : : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : : +- SubqueryAlias t2 + : : : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + : : +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + : : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : : +- SubqueryAlias t1 + : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [t3a#x,t3b#x,t3c#x,t3d#xL,t3e#x,t3f#x,t3g#x,t3h#x,t3i#x]) + : +- Project [cast(t3a#x as string) AS t3a#x, cast(t3b#x as smallint) AS t3b#x, cast(t3c#x as int) AS t3c#x, cast(t3d#xL as bigint) AS t3d#xL, cast(t3e#x as float) AS t3e#x, cast(t3f#x as double) AS t3f#x, cast(t3g#x as double) AS t3g#x, cast(t3h#x as timestamp) AS t3h#x, cast(t3i#x as date) AS t3i#x] + : +- Project [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + : +- SubqueryAlias t3 + : +- LocalRelation [t3a#x, t3b#x, t3c#x, t3d#xL, t3e#x, t3f#x, t3g#x, t3h#x, t3i#x] + +- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, + (SELECT count(t2d) FROM t2 WHERE t2a = t1a) count_t2, + (SELECT count_if(t2d > 0) FROM t2 WHERE t2a = t1a) count_if_t2, + (SELECT approx_count_distinct(t2d) FROM t2 WHERE t2a = t1a) approx_count_distinct_t2, + (SELECT collect_list(t2d) FROM t2 WHERE t2a = t1a) collect_list_t2, + (SELECT sort_array(collect_set(t2d)) FROM t2 WHERE t2a = t1a) collect_set_t2, + (SELECT hex(count_min_sketch(t2d, 0.5d, 0.5d, 1)) FROM t2 WHERE t2a = t1a) collect_set_t2 +FROM t1 +-- !query analysis +Project [t1a#x, scalar-subquery#x [t1a#x] AS count_t2#xL, scalar-subquery#x [t1a#x] AS count_if_t2#xL, scalar-subquery#x [t1a#x] AS approx_count_distinct_t2#xL, scalar-subquery#x [t1a#x] AS collect_list_t2#x, scalar-subquery#x [t1a#x] AS collect_set_t2#x, scalar-subquery#x [t1a#x] AS collect_set_t2#x] +: :- Aggregate [count(t2d#xL) AS count(t2d)#xL] +: : +- Filter (t2a#x = outer(t1a#x)) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: :- Aggregate [count_if((t2d#xL > cast(0 as bigint))) AS count_if((t2d > 0))#xL] +: : +- Filter (t2a#x = outer(t1a#x)) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: :- Aggregate [approx_count_distinct(t2d#xL, 0.05, 0, 0) AS approx_count_distinct(t2d)#xL] +: : +- Filter (t2a#x = outer(t1a#x)) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: :- Aggregate [collect_list(t2d#xL, 0, 0) AS collect_list(t2d)#x] +: : +- Filter (t2a#x = outer(t1a#x)) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: :- Aggregate [sort_array(collect_set(t2d#xL, 0, 0), true) AS sort_array(collect_set(t2d), true)#x] +: : +- Filter (t2a#x = outer(t1a#x)) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- Aggregate [hex(count_min_sketch(t2d#xL, 0.5, 0.5, 1, 0, 0)) AS hex(count_min_sketch(t2d, 0.5, 0.5, 1))#x] +: +- Filter (t2a#x = outer(t1a#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- SubqueryAlias t2 +: +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] ++- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1c, (SELECT t1c) FROM t1 +-- !query analysis +Project [t1c#x, scalar-subquery#x [t1c#x] AS scalarsubquery(t1c)#x] +: +- Project [outer(t1c#x)] +: +- OneRowRelation ++- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1c, (SELECT t1c WHERE t1c = 8) FROM t1 +-- !query analysis +Project [t1c#x, scalar-subquery#x [t1c#x && t1c#x] AS scalarsubquery(t1c, t1c)#x] +: +- Project [outer(t1c#x)] +: +- Filter (outer(t1c#x) = 8) +: +- OneRowRelation ++- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1c, t1d, (SELECT c + d FROM (SELECT t1c AS c, t1d AS d)) FROM t1 +-- !query analysis +Project [t1c#x, t1d#xL, scalar-subquery#x [t1c#x && t1d#xL] AS scalarsubquery(t1c, t1d)#xL] +: +- Project [(cast(c#x as bigint) + d#xL) AS (c + d)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [outer(t1c#x) AS c#x, outer(t1d#xL) AS d#xL] +: +- OneRowRelation ++- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1c, (SELECT SUM(c) FROM (SELECT t1c AS c)) FROM t1 +-- !query analysis +Project [t1c#x, scalar-subquery#x [t1c#x] AS scalarsubquery(t1c)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [outer(t1c#x) AS c#x] +: +- OneRowRelation ++- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +SELECT t1a, (SELECT SUM(t2b) FROM t2 JOIN (SELECT t1a AS a) ON t2a = a) FROM t1 +-- !query analysis +Project [t1a#x, scalar-subquery#x [t1a#x] AS scalarsubquery(t1a)#xL] +: +- Aggregate [sum(t2b#x) AS sum(t2b)#xL] +: +- Join Inner, (t2a#x = a#x) +: :- SubqueryAlias t2 +: : +- View (`t2`, [t2a#x,t2b#x,t2c#x,t2d#xL,t2e#x,t2f#x,t2g#x,t2h#x,t2i#x]) +: : +- Project [cast(t2a#x as string) AS t2a#x, cast(t2b#x as smallint) AS t2b#x, cast(t2c#x as int) AS t2c#x, cast(t2d#xL as bigint) AS t2d#xL, cast(t2e#x as float) AS t2e#x, cast(t2f#x as double) AS t2f#x, cast(t2g#x as double) AS t2g#x, cast(t2h#x as timestamp) AS t2h#x, cast(t2i#x as date) AS t2i#x] +: : +- Project [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [outer(t1a#x) AS a#x] +: +- OneRowRelation ++- SubqueryAlias t1 + +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) + +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] + +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + +- SubqueryAlias t1 + +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (0, 1), (1, 2) t1(c1, c2) +-- !query analysis +CreateViewCommand `t1`, VALUES (0, 1), (1, 2) t1(c1, c2), false, true, LocalTempView, true + +- SubqueryAlias t1 + +- LocalRelation [c1#x, c2#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (0, 2), (0, 3) t2(c1, c2) +-- !query analysis +CreateViewCommand `t2`, VALUES (0, 2), (0, 3) t2(c1, c2), false, true, LocalTempView, true + +- SubqueryAlias t2 + +- LocalRelation [c1#x, c2#x] + + +-- !query +SELECT c1, (WITH t AS (SELECT 1 AS a) SELECT a + c1 FROM t) FROM t1 +-- !query analysis +Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#x] +: +- WithCTE +: :- CTERelationDef xxxx, false +: : +- SubqueryAlias t +: : +- Project [1 AS a#x] +: : +- OneRowRelation +: +- Project [(a#x + outer(c1#x)) AS (a + outer(t1.c1))#x] +: +- SubqueryAlias t +: +- CTERelationRef xxxx, true, [a#x] ++- SubqueryAlias t1 + +- View (`t1`, [c1#x,c2#x]) + +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] + +- SubqueryAlias t1 + +- LocalRelation [c1#x, c2#x] + + +-- !query +SELECT c1, (WITH t AS (SELECT * FROM t2 WHERE c1 = t1.c1) SELECT SUM(c2) FROM t) FROM t1 +-- !query analysis +Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] +: +- WithCTE +: :- CTERelationDef xxxx, false +: : +- SubqueryAlias t +: : +- Project [c1#x, c2#x] +: : +- Filter (c1#x = outer(c1#x)) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [c1#x,c2#x]) +: : +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [c1#x, c2#x] +: +- Aggregate [sum(c2#x) AS sum(c2)#xL] +: +- SubqueryAlias t +: +- CTERelationRef xxxx, true, [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 int) AS c2#x] + +- SubqueryAlias t1 + +- LocalRelation [c1#x, c2#x] + + +-- !query +SELECT c1, ( + WITH t3 AS (SELECT c1 + 1 AS c1, c2 + 1 AS c2 FROM t2), + t4 AS (SELECT * FROM t3 WHERE t1.c1 = c1) + SELECT SUM(c2) FROM t4 +) FROM t1 +-- !query analysis +Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] +: +- WithCTE +: :- CTERelationDef xxxx, false +: : +- SubqueryAlias t3 +: : +- Project [(c1#x + 1) AS c1#x, (c2#x + 1) AS c2#x] +: : +- SubqueryAlias t2 +: : +- View (`t2`, [c1#x,c2#x]) +: : +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [c1#x, c2#x] +: :- CTERelationDef xxxx, false +: : +- SubqueryAlias t4 +: : +- Project [c1#x, c2#x] +: : +- Filter (outer(c1#x) = c1#x) +: : +- SubqueryAlias t3 +: : +- CTERelationRef xxxx, true, [c1#x, c2#x] +: +- Aggregate [sum(c2#x) AS sum(c2)#xL] +: +- SubqueryAlias t4 +: +- CTERelationRef xxxx, true, [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 int) AS c2#x] + +- SubqueryAlias t1 + +- LocalRelation [c1#x, c2#x] + + +-- !query +SELECT c1, ( + WITH t AS (SELECT * FROM t2) + SELECT SUM(c2) FROM (SELECT c1, c2 FROM t UNION SELECT c2, c1 FROM t) r(c1, c2) + WHERE c1 = t1.c1 +) FROM t1 +-- !query analysis +Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] +: +- WithCTE +: :- CTERelationDef xxxx, false +: : +- SubqueryAlias t +: : +- Project [c1#x, c2#x] +: : +- SubqueryAlias t2 +: : +- View (`t2`, [c1#x,c2#x]) +: : +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [c1#x, c2#x] +: +- Aggregate [sum(c2#x) AS sum(c2)#xL] +: +- Filter (c1#x = outer(c1#x)) +: +- SubqueryAlias r +: +- Project [c1#x AS c1#x, c2#x AS c2#x] +: +- Distinct +: +- Union false, false +: :- Project [c1#x, c2#x] +: : +- SubqueryAlias t +: : +- CTERelationRef xxxx, true, [c1#x, c2#x] +: +- Project [c2#x AS c2#x, c1#x AS c1#x] +: +- Project [c2#x, c1#x] +: +- SubqueryAlias t +: +- CTERelationRef xxxx, true, [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 int) AS c2#x] + +- SubqueryAlias t1 + +- LocalRelation [c1#x, c2#x] + + +-- !query +WITH v AS (SELECT * FROM t2) +SELECT * FROM t1 WHERE c1 > ( + WITH t AS (SELECT * FROM t2) + SELECT COUNT(*) FROM v WHERE c1 = t1.c1 AND c1 > (SELECT SUM(c2) FROM t WHERE c1 = v.c1) +) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias v +: +- Project [c1#x, c2#x] +: +- SubqueryAlias t2 +: +- View (`t2`, [c1#x,c2#x]) +: +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] +: +- SubqueryAlias t2 +: +- LocalRelation [c1#x, c2#x] ++- Project [c1#x, c2#x] + +- Filter (cast(c1#x as bigint) > scalar-subquery#x [c1#x]) + : +- WithCTE + : :- CTERelationDef xxxx, false + : : +- SubqueryAlias t + : : +- Project [c1#x, c2#x] + : : +- SubqueryAlias t2 + : : +- View (`t2`, [c1#x,c2#x]) + : : +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] + : : +- SubqueryAlias t2 + : : +- LocalRelation [c1#x, c2#x] + : +- Aggregate [count(1) AS count(1)#xL] + : +- Filter ((c1#x = outer(c1#x)) AND (cast(c1#x as bigint) > scalar-subquery#x [c1#x])) + : : +- Aggregate [sum(c2#x) AS sum(c2)#xL] + : : +- Filter (c1#x = outer(c1#x)) + : : +- SubqueryAlias t + : : +- CTERelationRef xxxx, true, [c1#x, c2#x] + : +- SubqueryAlias v + : +- CTERelationRef xxxx, true, [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 int) AS c2#x] + +- SubqueryAlias t1 + +- LocalRelation [c1#x, c2#x] + + +-- !query +WITH t AS (SELECT 1 AS a) +SELECT c1, (SELECT a FROM t WHERE a = c1) FROM t1 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1 AS a#x] +: +- OneRowRelation ++- Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#x] + : +- Project [a#x] + : +- Filter (a#x = outer(c1#x)) + : +- SubqueryAlias t + : +- CTERelationRef xxxx, true, [a#x] + +- SubqueryAlias t1 + +- View (`t1`, [c1#x,c2#x]) + +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] + +- SubqueryAlias t1 + +- LocalRelation [c1#x, c2#x] + + +-- !query +WITH +v1 AS (SELECT c1, c2, rand(0) c3 FROM t1), +v2 AS (SELECT c1, c2, rand(0) c4 FROM v1 WHERE c3 IN (SELECT c3 FROM v1)) +SELECT c1, ( + WITH v3 AS (SELECT c1, c2, rand(0) c5 FROM t2) + SELECT COUNT(*) FROM ( + SELECT * FROM v2 WHERE c1 > 0 + UNION SELECT * FROM v2 WHERE c2 > 0 + UNION SELECT * FROM v3 WHERE c2 > 0 + ) WHERE c1 = v1.c1 +) FROM v1 +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT (SELECT a FROM (SELECT 1 AS a UNION ALL SELECT 2 AS a) t) AS b +-- !query analysis +Project [scalar-subquery#x [] AS b#x] +: +- Project [a#x] +: +- SubqueryAlias t +: +- Union false, false +: :- Project [1 AS a#x] +: : +- OneRowRelation +: +- Project [2 AS a#x] +: +- OneRowRelation ++- OneRowRelation + + +-- !query +CREATE OR REPLACE TEMP VIEW t1(c1, c2) AS (VALUES (0, 1), (1, 2)) +-- !query analysis +CreateViewCommand `t1`, [(c1,None), (c2,None)], (VALUES (0, 1), (1, 2)), false, true, LocalTempView, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE OR REPLACE TEMP VIEW t2(c1, c2) AS (VALUES (0, 2), (0, 3)) +-- !query analysis +CreateViewCommand `t2`, [(c1,None), (c2,None)], (VALUES (0, 2), (0, 3)), false, true, LocalTempView, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE OR REPLACE TEMP VIEW students(id, name, major, year) AS (VALUES + (0, 'A', 'CS', 2022), + (1, 'B', 'CS', 2022), + (2, 'C', 'Math', 2022)) +-- !query analysis +CreateViewCommand `students`, [(id,None), (name,None), (major,None), (year,None)], (VALUES + (0, 'A', 'CS', 2022), + (1, 'B', 'CS', 2022), + (2, 'C', 'Math', 2022)), false, true, LocalTempView, true + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +CREATE OR REPLACE TEMP VIEW exams(sid, course, curriculum, grade, date) AS (VALUES + (0, 'C1', 'CS', 4, 2020), + (0, 'C2', 'CS', 3, 2021), + (1, 'C1', 'CS', 2, 2020), + (1, 'C2', 'CS', 1, 2021)) +-- !query analysis +CreateViewCommand `exams`, [(sid,None), (course,None), (curriculum,None), (grade,None), (date,None)], (VALUES + (0, 'C1', 'CS', 4, 2020), + (0, 'C2', 'CS', 3, 2021), + (1, 'C1', 'CS', 2, 2020), + (1, 'C2', 'CS', 1, 2021)), false, true, LocalTempView, true + +- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x] + + +-- !query +SELECT students.name, exams.course +FROM students, exams +WHERE students.id = exams.sid + AND (students.major = 'CS' OR students.major = 'Games Eng') + AND exams.grade >= ( + SELECT avg(exams.grade) + 1 + FROM exams + WHERE students.id = exams.sid + OR (exams.curriculum = students.major AND students.year > exams.date)) +-- !query analysis +Project [name#x, course#x] ++- Filter (((id#x = sid#x) AND ((major#x = CS) OR (major#x = Games Eng))) AND (cast(grade#x as double) >= scalar-subquery#x [id#x && major#x && year#x])) + : +- Aggregate [(avg(grade#x) + cast(1 as double)) AS (avg(grade) + 1)#x] + : +- Filter ((outer(id#x) = sid#x) OR ((curriculum#x = outer(major#x)) AND (outer(year#x) > date#x))) + : +- SubqueryAlias exams + : +- View (`exams`, [sid#x,course#x,curriculum#x,grade#x,date#x]) + : +- Project [cast(col1#x as int) AS sid#x, cast(col2#x as string) AS course#x, cast(col3#x as string) AS curriculum#x, cast(col4#x as int) AS grade#x, cast(col5#x as int) AS date#x] + : +- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x] + +- Join Inner + :- SubqueryAlias students + : +- View (`students`, [id#x,name#x,major#x,year#x]) + : +- Project [cast(col1#x as int) AS id#x, cast(col2#x as string) AS name#x, cast(col3#x as string) AS major#x, cast(col4#x as int) AS year#x] + : +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + +- SubqueryAlias exams + +- View (`exams`, [sid#x,course#x,curriculum#x,grade#x,date#x]) + +- Project [cast(col1#x as int) AS sid#x, cast(col2#x as string) AS course#x, cast(col3#x as string) AS curriculum#x, cast(col4#x as int) AS grade#x, cast(col5#x as int) AS date#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x] + + +-- !query +SELECT (SELECT min(c2) FROM t2 WHERE t1.c1 > t2.c1) FROM t1 +-- !query analysis +Project [scalar-subquery#x [c1#x] AS scalarsubquery(c1)#x] +: +- Aggregate [min(c2#x) AS min(c2)#x] +: +- Filter (outer(c1#x) > c1#x) +: +- SubqueryAlias t2 +: +- View (`t2`, [c1#x,c2#x]) +: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] +: +- LocalRelation [col1#x, col2#x] ++- SubqueryAlias t1 + +- View (`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT (SELECT min(c2) FROM t2 WHERE t1.c1 >= t2.c1 AND t1.c2 < t2.c2) FROM t1 +-- !query analysis +Project [scalar-subquery#x [c1#x && c2#x] AS scalarsubquery(c1, c2)#x] +: +- Aggregate [min(c2#x) AS min(c2)#x] +: +- Filter ((outer(c1#x) >= c1#x) AND (outer(c2#x) < c2#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [c1#x,c2#x]) +: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] +: +- LocalRelation [col1#x, col2#x] ++- SubqueryAlias t1 + +- View (`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT (SELECT count(*) FROM t2 WHERE t1.c1 > t2.c1) FROM t1 +-- !query analysis +Project [scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] +: +- Aggregate [count(1) AS count(1)#xL] +: +- Filter (outer(c1#x) > c1#x) +: +- SubqueryAlias t2 +: +- View (`t2`, [c1#x,c2#x]) +: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] +: +- LocalRelation [col1#x, col2#x] ++- SubqueryAlias t1 + +- View (`t1`, [c1#x,c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT c, ( + SELECT count(*) + FROM (VALUES ('ab'), ('abc'), ('bc')) t2(c) + WHERE t1.c = substring(t2.c, 1, 1) +) FROM (VALUES ('a'), ('b')) t1(c) +-- !query analysis +Project [c#x, scalar-subquery#x [c#x] AS scalarsubquery(c)#xL] +: +- Aggregate [count(1) AS count(1)#xL] +: +- Filter (outer(c#x) = substring(c#x, 1, 1)) +: +- SubqueryAlias t2 +: +- Project [col1#x AS c#x] +: +- LocalRelation [col1#x] ++- SubqueryAlias t1 + +- Project [col1#x AS c#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT c, ( + SELECT count(*) + FROM (VALUES (0, 6), (1, 5), (2, 4), (3, 3)) t1(a, b) + WHERE a + b = c +) FROM (VALUES (6)) t2(c) +-- !query analysis +Project [c#x, scalar-subquery#x [c#x] AS scalarsubquery(c)#xL] +: +- Aggregate [count(1) AS count(1)#xL] +: +- Filter ((a#x + b#x) = outer(c#x)) +: +- SubqueryAlias t1 +: +- Project [col1#x AS a#x, col2#x AS b#x] +: +- LocalRelation [col1#x, col2#x] ++- SubqueryAlias t2 + +- Project [col1#x AS c#x] + +- LocalRelation [col1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out new file mode 100644 index 0000000000000..0f53d05ac3e7d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out @@ -0,0 +1,1804 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMP VIEW t0(t0a, t0b) AS VALUES (1, 1), (2, 0) +-- !query analysis +CreateViewCommand `t0`, [(t0a,None), (t0b,None)], VALUES (1, 1), (2, 0), false, true, LocalTempView, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE OR REPLACE TEMP VIEW t1(t1a, t1b, t1c) AS VALUES (1, 1, 3) +-- !query analysis +CreateViewCommand `t1`, [(t1a,None), (t1b,None), (t1c,None)], VALUES (1, 1, 3), false, true, LocalTempView, true + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +CREATE OR REPLACE TEMP VIEW t2(t2a, t2b, t2c) AS VALUES (1, 1, 5), (2, 2, 7) +-- !query analysis +CreateViewCommand `t2`, [(t2a,None), (t2b,None), (t2c,None)], VALUES (1, 1, 5), (2, 2, 7), false, true, LocalTempView, true + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Union false, false +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +-- !query analysis +Project [t0a#x, t0b#x] ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0b#x]) + : +- Aggregate [sum(c#x) AS sum(c)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Union false, false + : :- Project [t1c#x AS c#x] + : : +- Filter (t1a#x = outer(t0a#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) + : : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] + : : +- LocalRelation [col1#x, col2#x, col3#x] + : +- Project [t2c#x AS c#x] + : +- Filter (t2b#x = outer(t0b#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x]) + : +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Union false, false +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2a#x = outer(t0a#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Union false, false +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x > outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2b#x <= outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(t1c#x) AS sum(t1c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Union false, false +: :- Project [t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] +: +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Union false, false +: :- Project [t1c#x AS t1a#x, t1a#x AS t1b#x, outer(t0a#x) AS t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [outer(t0a#x) AS t2b#x, t2c#x AS t1a#x, outer(t0b#x) AS t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [count(t1c#x) AS count(t1c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Union false, false +: :- Project [t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + UNION ALL + SELECT t2a - t0a as d + FROM t2) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +: +- Aggregate [sum(d#x) AS sum(d)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Union false, false +: :- Project [(t1a#x - outer(t0a#x)) AS d#x] +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [(t2a#x - outer(t0a#x)) AS d#x] +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + UNION ALL + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(t0a) AS d\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 67, + "fragment" : "SELECT sum(t0a) as d\n FROM t1" + } ] +} + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Distinct +: +- Union false, false +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +-- !query analysis +Project [t0a#x, t0b#x] ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0b#x]) + : +- Aggregate [sum(c#x) AS sum(c)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Distinct + : +- Union false, false + : :- Project [t1c#x AS c#x] + : : +- Filter (t1a#x = outer(t0a#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) + : : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] + : : +- LocalRelation [col1#x, col2#x, col3#x] + : +- Project [t2c#x AS c#x] + : +- Filter (t2b#x = outer(t0b#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x]) + : +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Distinct +: +- Union false, false +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2a#x = outer(t0a#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + UNION DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Distinct +: +- Union false, false +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x > outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2b#x <= outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(t1c#x) AS sum(t1c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Distinct +: +- Union false, false +: :- Project [t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] +: +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Distinct +: +- Union false, false +: :- Project [t1c#x AS t1a#x, t1a#x AS t1b#x, outer(t0a#x) AS t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [outer(t0a#x) AS t2b#x, t2c#x AS t1a#x, outer(t0b#x) AS t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [count(t1c#x) AS count(t1c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Distinct +: +- Union false, false +: :- Project [t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + UNION DISTINCT + SELECT t2a - t0a as d + FROM t2) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +: +- Aggregate [sum(d#x) AS sum(d)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Distinct +: +- Union false, false +: :- Project [(t1a#x - outer(t0a#x)) AS d#x] +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [(t2a#x - outer(t0a#x)) AS d#x] +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + UNION DISTINCT + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(t0a) AS d\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 67, + "fragment" : "SELECT sum(t0a) as d\n FROM t1" + } ] +} + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Intersect All true +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +-- !query analysis +Project [t0a#x, t0b#x] ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0b#x]) + : +- Aggregate [sum(c#x) AS sum(c)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Intersect All true + : :- Project [t1c#x AS c#x] + : : +- Filter (t1a#x = outer(t0a#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) + : : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] + : : +- LocalRelation [col1#x, col2#x, col3#x] + : +- Project [t2c#x AS c#x] + : +- Filter (t2b#x = outer(t0b#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x]) + : +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Intersect All true +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2a#x = outer(t0a#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + INTERSECT ALL + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Intersect All true +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x > outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2b#x <= outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(t1c#x) AS sum(t1c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Intersect All true +: :- Project [t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] +: +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Intersect All true +: :- Project [t1c#x AS t1a#x, t1a#x AS t1b#x, outer(t0a#x) AS t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [outer(t0a#x) AS t2b#x, t2c#x AS t1a#x, outer(t0b#x) AS t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [count(t1c#x) AS count(t1c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Intersect All true +: :- Project [t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + INTERSECT ALL + SELECT t2a - t0a as d + FROM t2) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +: +- Aggregate [sum(d#x) AS sum(d)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Intersect All true +: :- Project [(t1a#x - outer(t0a#x)) AS d#x] +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [(t2a#x - outer(t0a#x)) AS d#x] +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + INTERSECT ALL + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(t0a) AS d\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 67, + "fragment" : "SELECT sum(t0a) as d\n FROM t1" + } ] +} + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Intersect false +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +-- !query analysis +Project [t0a#x, t0b#x] ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0b#x]) + : +- Aggregate [sum(c#x) AS sum(c)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Intersect false + : :- Project [t1c#x AS c#x] + : : +- Filter (t1a#x = outer(t0a#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) + : : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] + : : +- LocalRelation [col1#x, col2#x, col3#x] + : +- Project [t2c#x AS c#x] + : +- Filter (t2b#x = outer(t0b#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x]) + : +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Intersect false +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2a#x = outer(t0a#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + INTERSECT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Intersect false +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x > outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2b#x <= outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(t1c#x) AS sum(t1c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Intersect false +: :- Project [t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] +: +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Intersect false +: :- Project [t1c#x AS t1a#x, t1a#x AS t1b#x, outer(t0a#x) AS t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [outer(t0a#x) AS t2b#x, t2c#x AS t1a#x, outer(t0b#x) AS t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [count(t1c#x) AS count(t1c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Intersect false +: :- Project [t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + INTERSECT DISTINCT + SELECT t2a - t0a as d + FROM t2) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +: +- Aggregate [sum(d#x) AS sum(d)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Intersect false +: :- Project [(t1a#x - outer(t0a#x)) AS d#x] +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [(t2a#x - outer(t0a#x)) AS d#x] +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + INTERSECT DISTINCT + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(t0a) AS d\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 67, + "fragment" : "SELECT sum(t0a) as d\n FROM t1" + } ] +} + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Except All true +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +-- !query analysis +Project [t0a#x, t0b#x] ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0b#x]) + : +- Aggregate [sum(c#x) AS sum(c)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Except All true + : :- Project [t1c#x AS c#x] + : : +- Filter (t1a#x = outer(t0a#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) + : : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] + : : +- LocalRelation [col1#x, col2#x, col3#x] + : +- Project [t2c#x AS c#x] + : +- Filter (t2b#x = outer(t0b#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x]) + : +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Except All true +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2a#x = outer(t0a#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + EXCEPT ALL + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Except All true +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x > outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2b#x <= outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(t1c#x) AS sum(t1c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Except All true +: :- Project [t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] +: +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Except All true +: :- Project [t1c#x AS t1a#x, t1a#x AS t1b#x, outer(t0a#x) AS t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [outer(t0a#x) AS t2b#x, t2c#x AS t1a#x, outer(t0b#x) AS t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [count(t1c#x) AS count(t1c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Except All true +: :- Project [t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + EXCEPT ALL + SELECT t2a - t0a as d + FROM t2) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +: +- Aggregate [sum(d#x) AS sum(d)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Except All true +: :- Project [(t1a#x - outer(t0a#x)) AS d#x] +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [(t2a#x - outer(t0a#x)) AS d#x] +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + EXCEPT ALL + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(t0a) AS d\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 67, + "fragment" : "SELECT sum(t0a) as d\n FROM t1" + } ] +} + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Except false +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +-- !query analysis +Project [t0a#x, t0b#x] ++- Filter (cast(t0a#x as bigint) < scalar-subquery#x [t0a#x && t0b#x]) + : +- Aggregate [sum(c#x) AS sum(c)#xL] + : +- SubqueryAlias __auto_generated_subquery_name + : +- Except false + : :- Project [t1c#x AS c#x] + : : +- Filter (t1a#x = outer(t0a#x)) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) + : : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] + : : +- LocalRelation [col1#x, col2#x, col3#x] + : +- Project [t2c#x AS c#x] + : +- Filter (t2b#x = outer(t0b#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [t2a#x,t2b#x,t2c#x]) + : +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Except false +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2a#x = outer(t0a#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + EXCEPT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(c#x) AS sum(c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Except false +: :- Project [t1c#x AS c#x] +: : +- Filter (t1a#x > outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x AS c#x] +: +- Filter (t2b#x <= outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [sum(t1c#x) AS sum(t1c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Except false +: :- Project [t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x && t0a#x && t0b#x && t0b#x] AS scalarsubquery(t0a, t0a, t0a, t0b, t0b)#xL] +: +- Aggregate [sum(((t1a#x + (3 * t1b#x)) + (5 * t1c#x))) AS sum(((t1a + (3 * t1b)) + (5 * t1c)))#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Except false +: :- Project [t1c#x AS t1a#x, t1a#x AS t1b#x, outer(t0a#x) AS t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [outer(t0a#x) AS t2b#x, t2c#x AS t1a#x, outer(t0b#x) AS t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0b#x] AS scalarsubquery(t0a, t0b)#xL] +: +- Aggregate [count(t1c#x) AS count(t1c)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Except false +: :- Project [t1c#x] +: : +- Filter (t1a#x = outer(t0a#x)) +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [t2c#x] +: +- Filter (t2b#x = outer(t0b#x)) +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + EXCEPT DISTINCT + SELECT t2a - t0a as d + FROM t2) +) +FROM t0 +-- !query analysis +Project [t0a#x, scalar-subquery#x [t0a#x && t0a#x] AS scalarsubquery(t0a, t0a)#xL] +: +- Aggregate [sum(d#x) AS sum(d)#xL] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Except false +: :- Project [(t1a#x - outer(t0a#x)) AS d#x] +: : +- SubqueryAlias t1 +: : +- View (`t1`, [t1a#x,t1b#x,t1c#x]) +: : +- Project [cast(col1#x as int) AS t1a#x, cast(col2#x as int) AS t1b#x, cast(col3#x as int) AS t1c#x] +: : +- LocalRelation [col1#x, col2#x, col3#x] +: +- Project [(t2a#x - outer(t0a#x)) AS d#x] +: +- SubqueryAlias t2 +: +- View (`t2`, [t2a#x,t2b#x,t2c#x]) +: +- Project [cast(col1#x as int) AS t2a#x, cast(col2#x as int) AS t2b#x, cast(col3#x as int) AS t2c#x] +: +- LocalRelation [col1#x, col2#x, col3#x] ++- SubqueryAlias t0 + +- View (`t0`, [t0a#x,t0b#x]) + +- Project [cast(col1#x as int) AS t0a#x, cast(col2#x as int) AS t0b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + EXCEPT DISTINCT + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(t0a) AS d\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 67, + "fragment" : "SELECT sum(t0a) as d\n FROM t1" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/subquery-in-from.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/subquery-in-from.sql.out new file mode 100644 index 0000000000000..2a26c901624c3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/subquery-in-from.sql.out @@ -0,0 +1,65 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT * FROM (SELECT * FROM testData) AS t WHERE key = 1 +-- !query analysis +Project [key#x, value#x] ++- Filter (key#x = 1) + +- SubqueryAlias t + +- Project [key#x, value#x] + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +FROM (SELECT * FROM testData WHERE key = 1) AS t SELECT * +-- !query analysis +Project [key#x, value#x] ++- SubqueryAlias t + +- Project [key#x, value#x] + +- Filter (key#x = 1) + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +SELECT * FROM (SELECT * FROM testData) t WHERE key = 1 +-- !query analysis +Project [key#x, value#x] ++- Filter (key#x = 1) + +- SubqueryAlias t + +- Project [key#x, value#x] + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +FROM (SELECT * FROM testData WHERE key = 1) t SELECT * +-- !query analysis +Project [key#x, value#x] ++- SubqueryAlias t + +- Project [key#x, value#x] + +- Filter (key#x = 1) + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +SELECT * FROM (SELECT * FROM testData) WHERE key = 1 +-- !query analysis +Project [key#x, value#x] ++- Filter (key#x = 1) + +- SubqueryAlias __auto_generated_subquery_name + +- Project [key#x, value#x] + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet + + +-- !query +FROM (SELECT * FROM testData WHERE key = 1) SELECT * +-- !query analysis +Project [key#x, value#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [key#x, value#x] + +- Filter (key#x = 1) + +- SubqueryAlias spark_catalog.default.testdata + +- Relation spark_catalog.default.testdata[key#x,value#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/table-aliases.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/table-aliases.sql.out new file mode 100644 index 0000000000000..20933abe3b1f3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/table-aliases.sql.out @@ -0,0 +1,217 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1) AS testData(a, b) +-- !query analysis +CreateViewCommand `testData`, SELECT * FROM VALUES (1, 1), (1, 2), (2, 1) AS testData(a, b), false, true, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * FROM testData AS t(col1, col2) WHERE col1 = 1 +-- !query analysis +Project [col1#x, col2#x] ++- Filter (col1#x = 1) + +- SubqueryAlias t + +- Project [a#x AS col1#x, b#x AS col2#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * FROM testData AS t(col1, col2) WHERE col1 = 2 +-- !query analysis +Project [col1#x, col2#x] ++- Filter (col1#x = 2) + +- SubqueryAlias t + +- Project [a#x AS col1#x, b#x AS col2#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT col1 AS k, SUM(col2) FROM testData AS t(col1, col2) GROUP BY k +-- !query analysis +Aggregate [col1#x], [col1#x AS k#x, sum(col2#x) AS sum(col2)#xL] ++- SubqueryAlias t + +- Project [a#x AS col1#x, b#x AS col2#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT * FROM testData AS t(col1, col2, col3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1028", + "messageParameters" : { + "columnSize" : "3", + "outputSize" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 45, + "fragment" : "testData AS t(col1, col2, col3)" + } ] +} + + +-- !query +SELECT * FROM testData AS t(col1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1028", + "messageParameters" : { + "columnSize" : "1", + "outputSize" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 33, + "fragment" : "testData AS t(col1)" + } ] +} + + +-- !query +SELECT a AS col1, b AS col2 FROM testData AS t(c, d) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`", + "proposal" : "`t`.`c`, `t`.`d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 8, + "fragment" : "a" + } ] +} + + +-- !query +SELECT * FROM (SELECT 1 AS a, 1 AS b) t(col1, col2) +-- !query analysis +Project [col1#x, col2#x] ++- SubqueryAlias t + +- Project [a#x AS col1#x, b#x AS col2#x] + +- Project [1 AS a#x, 1 AS b#x] + +- OneRowRelation + + +-- !query +SELECT t.* FROM (SELECT 1 AS a, 1 AS b) t(col1, col2) +-- !query analysis +Project [col1#x, col2#x] ++- SubqueryAlias t + +- Project [a#x AS col1#x, b#x AS col2#x] + +- Project [1 AS a#x, 1 AS b#x] + +- OneRowRelation + + +-- !query +SELECT col1, col2 FROM (SELECT 1 AS a, 1 AS b) t(col1, col2) +-- !query analysis +Project [col1#x, col2#x] ++- SubqueryAlias t + +- Project [a#x AS col1#x, b#x AS col2#x] + +- Project [1 AS a#x, 1 AS b#x] + +- OneRowRelation + + +-- !query +SELECT t.col1, t.col2 FROM (SELECT 1 AS a, 1 AS b) t(col1, col2) +-- !query analysis +Project [col1#x, col2#x] ++- SubqueryAlias t + +- Project [a#x AS col1#x, b#x AS col2#x] + +- Project [1 AS a#x, 1 AS b#x] + +- OneRowRelation + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW src1 AS SELECT * FROM VALUES (1, "a"), (2, "b"), (3, "c") AS src1(id, v1) +-- !query analysis +CreateViewCommand `src1`, SELECT * FROM VALUES (1, "a"), (2, "b"), (3, "c") AS src1(id, v1), false, true, LocalTempView, true + +- Project [id#x, v1#x] + +- SubqueryAlias src1 + +- LocalRelation [id#x, v1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW src2 AS SELECT * FROM VALUES (2, 1.0), (3, 3.2), (1, 8.5) AS src2(id, v2) +-- !query analysis +CreateViewCommand `src2`, SELECT * FROM VALUES (2, 1.0), (3, 3.2), (1, 8.5) AS src2(id, v2), false, true, LocalTempView, true + +- Project [id#x, v2#x] + +- SubqueryAlias src2 + +- LocalRelation [id#x, v2#x] + + +-- !query +SELECT * FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d) +-- !query analysis +Project [a#x, b#x, c#x, d#x] ++- SubqueryAlias dst + +- Project [id#x AS a#x, v1#x AS b#x, id#x AS c#x, v2#x AS d#x] + +- Join Inner, (id#x = id#x) + :- SubqueryAlias s1 + : +- SubqueryAlias src1 + : +- View (`src1`, [id#x,v1#x]) + : +- Project [cast(id#x as int) AS id#x, cast(v1#x as string) AS v1#x] + : +- Project [id#x, v1#x] + : +- SubqueryAlias src1 + : +- LocalRelation [id#x, v1#x] + +- SubqueryAlias s2 + +- SubqueryAlias src2 + +- View (`src2`, [id#x,v2#x]) + +- Project [cast(id#x as int) AS id#x, cast(v2#x as decimal(2,1)) AS v2#x] + +- Project [id#x, v2#x] + +- SubqueryAlias src2 + +- LocalRelation [id#x, v2#x] + + +-- !query +SELECT dst.* FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d) +-- !query analysis +Project [a#x, b#x, c#x, d#x] ++- SubqueryAlias dst + +- Project [id#x AS a#x, v1#x AS b#x, id#x AS c#x, v2#x AS d#x] + +- Join Inner, (id#x = id#x) + :- SubqueryAlias s1 + : +- SubqueryAlias src1 + : +- View (`src1`, [id#x,v1#x]) + : +- Project [cast(id#x as int) AS id#x, cast(v1#x as string) AS v1#x] + : +- Project [id#x, v1#x] + : +- SubqueryAlias src1 + : +- LocalRelation [id#x, v1#x] + +- SubqueryAlias s2 + +- SubqueryAlias src2 + +- View (`src2`, [id#x,v2#x]) + +- Project [cast(id#x as int) AS id#x, cast(v2#x as decimal(2,1)) AS v2#x] + +- Project [id#x, v2#x] + +- SubqueryAlias src2 + +- LocalRelation [id#x, v2#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/table-valued-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/table-valued-functions.sql.out new file mode 100644 index 0000000000000..3342d5672c58f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/table-valued-functions.sql.out @@ -0,0 +1,927 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select * from dummy(3) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2308", + "messageParameters" : { + "name" : "dummy" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 22, + "fragment" : "dummy(3)" + } ] +} + + +-- !query +select * from range(6 + cos(3)) +-- !query analysis +Project [id#xL] ++- Range (0, 5, step=1, splits=None) + + +-- !query +select * from range(5, 10) +-- !query analysis +Project [id#xL] ++- Range (5, 10, step=1, splits=None) + + +-- !query +select * from range(0, 10, 2) +-- !query analysis +Project [id#xL] ++- Range (0, 10, step=2, splits=None) + + +-- !query +select * from range(0, 10, 1, 200) +-- !query analysis +Project [id#xL] ++- Range (0, 10, step=1, splits=Some(200)) + + +-- !query +select * from range(1, 1, 1, 1, 1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "5", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[1, 2, 3, 4]", + "functionName" : "`range`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 34, + "fragment" : "range(1, 1, 1, 1, 1)" + } ] +} + + +-- !query +select * from range(1, null) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`range`", + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "2", + "requiredType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 28, + "fragment" : "range(1, null)" + } ] +} + + +-- !query +select * from range(array(1, 2, 3)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`range`", + "inputSql" : "\"array(1, 2, 3)\"", + "inputType" : "\"ARRAY\"", + "paramIndex" : "2", + "requiredType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 35, + "fragment" : "range(array(1, 2, 3))" + } ] +} + + +-- !query +select * from range(0, 5, 0) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "FAILED_FUNCTION_CALL", + "sqlState" : "38000", + "messageParameters" : { + "funcName" : "`range`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 28, + "fragment" : "range(0, 5, 0)" + } ] +} + + +-- !query +select * from RaNgE(2) +-- !query analysis +Project [id#xL] ++- Range (0, 2, step=1, splits=None) + + +-- !query +select i from range(0, 2) t(i) +-- !query analysis +Project [i#xL] ++- SubqueryAlias t + +- Project [id#xL AS i#xL] + +- Range (0, 2, step=1, splits=None) + + +-- !query +select * from explode(array(1, 2)) +-- !query analysis +Project [col#x] ++- Generate explode(array(1, 2)), false, [col#x] + +- OneRowRelation + + +-- !query +select * from explode(map('a', 1, 'b', 2)) +-- !query analysis +Project [key#x, value#x] ++- Generate explode(map(a, 1, b, 2)), false, [key#x, value#x] + +- OneRowRelation + + +-- !query +select * from explode(array()) +-- !query analysis +Project [col#x] ++- Generate explode(array()), false, [col#x] + +- OneRowRelation + + +-- !query +select * from explode(map()) +-- !query analysis +Project [key#x, value#x] ++- Generate explode(map()), false, [key#x, value#x] + +- OneRowRelation + + +-- !query +select * from explode(array(1, 2)) t(c1) +-- !query analysis +Project [c1#x] ++- SubqueryAlias t + +- Project [col#x AS c1#x] + +- Generate explode(array(1, 2)), false, [col#x] + +- OneRowRelation + + +-- !query +select * from explode(map('a', 1, 'b', 2)) t(k, v) +-- !query analysis +Project [k#x, v#x] ++- SubqueryAlias t + +- Project [key#x AS k#x, value#x AS v#x] + +- Generate explode(map(a, 1, b, 2)), false, [key#x, value#x] + +- OneRowRelation + + +-- !query +select * from explode(null) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 27, + "fragment" : "explode(null)" + } ] +} + + +-- !query +select * from explode(null) t(c1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 33, + "fragment" : "explode(null) t(c1)" + } ] +} + + +-- !query +select * from explode(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 24, + "fragment" : "explode(1)" + } ] +} + + +-- !query +select * from explode(1, 2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`explode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 27, + "fragment" : "explode(1, 2)" + } ] +} + + +-- !query +select * from explode(explode(array(1))) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "0A000", + "messageParameters" : { + "expression" : "\"explode(explode(array(1)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 40, + "fragment" : "explode(explode(array(1)))" + } ] +} + + +-- !query +select * from explode(array(1, 2)) t(c1, c2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2307", + "messageParameters" : { + "aliasesNum" : "2", + "funcName" : "explode", + "outColsNum" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 44, + "fragment" : "explode(array(1, 2)) t(c1, c2)" + } ] +} + + +-- !query +select * from explode_outer(array(1, 2)) +-- !query analysis +Project [col#x] ++- Generate explode(array(1, 2)), true, [col#x] + +- OneRowRelation + + +-- !query +select * from explode_outer(map('a', 1, 'b', 2)) +-- !query analysis +Project [key#x, value#x] ++- Generate explode(map(a, 1, b, 2)), true, [key#x, value#x] + +- OneRowRelation + + +-- !query +select * from explode_outer(array()) +-- !query analysis +Project [col#x] ++- Generate explode(array()), true, [col#x] + +- OneRowRelation + + +-- !query +select * from explode_outer(map()) +-- !query analysis +Project [key#x, value#x] ++- Generate explode(map()), true, [key#x, value#x] + +- OneRowRelation + + +-- !query +select * from range(2) join explode(array(1, 2)) +-- !query analysis +Project [id#xL, col#x] ++- Join Inner + :- Range (0, 2, step=1, splits=None) + +- Generate explode(array(1, 2)), false, [col#x] + +- OneRowRelation + + +-- !query +select * from range(2) join explode_outer(array()) +-- !query analysis +Project [id#xL, col#x] ++- Join Inner + :- Range (0, 2, step=1, splits=None) + +- Generate explode(array()), true, [col#x] + +- OneRowRelation + + +-- !query +select * from inline(array(struct(1, 'a'), struct(2, 'b'))) +-- !query analysis +Project [col1#x, col2#x] ++- Generate inline(array(struct(col1, 1, col2, a), struct(col1, 2, col2, b))), false, [col1#x, col2#x] + +- OneRowRelation + + +-- !query +select * from inline(array(struct(1, 'a'), struct(2, 'b'))) t(x, y) +-- !query analysis +Project [x#x, y#x] ++- SubqueryAlias t + +- Project [col1#x AS x#x, col2#x AS y#x] + +- Generate inline(array(struct(col1, 1, col2, a), struct(col1, 2, col2, b))), false, [col1#x, col2#x] + +- OneRowRelation + + +-- !query +select * from inline(array_remove(array(struct(1, 'a')), struct(1, 'a'))) +-- !query analysis +Project [col1#x, col2#x] ++- Generate inline(array_remove(array(struct(col1, 1, col2, a)), struct(col1, 1, col2, a))), false, [col1#x, col2#x] + +- OneRowRelation + + +-- !query +select * from inline(null) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "1", + "requiredType" : "\"ARRAY\"", + "sqlExpr" : "\"inline(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 26, + "fragment" : "inline(null)" + } ] +} + + +-- !query +select * from inline(array(struct(1, 2), struct(2, 3))) t(a, b, c) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2307", + "messageParameters" : { + "aliasesNum" : "3", + "funcName" : "inline", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 66, + "fragment" : "inline(array(struct(1, 2), struct(2, 3))) t(a, b, c)" + } ] +} + + +-- !query +select * from inline_outer(array(struct(1, 'a'), struct(2, 'b'))) +-- !query analysis +Project [col1#x, col2#x] ++- Generate inline(array(struct(col1, 1, col2, a), struct(col1, 2, col2, b))), true, [col1#x, col2#x] + +- OneRowRelation + + +-- !query +select * from inline_outer(array_remove(array(struct(1, 'a')), struct(1, 'a'))) +-- !query analysis +Project [col1#x, col2#x] ++- Generate inline(array_remove(array(struct(col1, 1, col2, a)), struct(col1, 1, col2, a))), true, [col1#x, col2#x] + +- OneRowRelation + + +-- !query +select * from posexplode(array()) +-- !query analysis +Project [pos#x, col#x] ++- Generate posexplode(array()), false, [pos#x, col#x] + +- OneRowRelation + + +-- !query +select * from posexplode(array(1, 2)) +-- !query analysis +Project [pos#x, col#x] ++- Generate posexplode(array(1, 2)), false, [pos#x, col#x] + +- OneRowRelation + + +-- !query +select * from posexplode(array(1, 2)) t(pos, x) +-- !query analysis +Project [pos#x, x#x] ++- SubqueryAlias t + +- Project [pos#x AS pos#x, col#x AS x#x] + +- Generate posexplode(array(1, 2)), false, [pos#x, col#x] + +- OneRowRelation + + +-- !query +select * from posexplode(map()) +-- !query analysis +Project [pos#x, key#x, value#x] ++- Generate posexplode(map()), false, [pos#x, key#x, value#x] + +- OneRowRelation + + +-- !query +select * from posexplode(map('a', 1, 'b', 2)) +-- !query analysis +Project [pos#x, key#x, value#x] ++- Generate posexplode(map(a, 1, b, 2)), false, [pos#x, key#x, value#x] + +- OneRowRelation + + +-- !query +select * from posexplode(map('a', 1, 'b', 2)) t(pos, k, v) +-- !query analysis +Project [pos#x, k#x, v#x] ++- SubqueryAlias t + +- Project [pos#x AS pos#x, key#x AS k#x, value#x AS v#x] + +- Generate posexplode(map(a, 1, b, 2)), false, [pos#x, key#x, value#x] + +- OneRowRelation + + +-- !query +select * from posexplode(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"posexplode(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 27, + "fragment" : "posexplode(1)" + } ] +} + + +-- !query +select * from posexplode(1, 2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`posexplode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 30, + "fragment" : "posexplode(1, 2)" + } ] +} + + +-- !query +select * from posexplode(explode(array(1))) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "0A000", + "messageParameters" : { + "expression" : "\"posexplode(explode(array(1)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 43, + "fragment" : "posexplode(explode(array(1)))" + } ] +} + + +-- !query +select * from posexplode(array(1, 2)) t(x) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2307", + "messageParameters" : { + "aliasesNum" : "1", + "funcName" : "posexplode", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 42, + "fragment" : "posexplode(array(1, 2)) t(x)" + } ] +} + + +-- !query +select * from posexplode_outer(array()) +-- !query analysis +Project [pos#x, col#x] ++- Generate posexplode(array()), true, [pos#x, col#x] + +- OneRowRelation + + +-- !query +select * from posexplode_outer(array(1, 2)) +-- !query analysis +Project [pos#x, col#x] ++- Generate posexplode(array(1, 2)), true, [pos#x, col#x] + +- OneRowRelation + + +-- !query +select * from posexplode_outer(map()) +-- !query analysis +Project [pos#x, key#x, value#x] ++- Generate posexplode(map()), true, [pos#x, key#x, value#x] + +- OneRowRelation + + +-- !query +select * from posexplode_outer(map('a', 1, 'b', 2)) +-- !query analysis +Project [pos#x, key#x, value#x] ++- Generate posexplode(map(a, 1, b, 2)), true, [pos#x, key#x, value#x] + +- OneRowRelation + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') +-- !query analysis +Project [c0#x, c1#x] ++- Generate json_tuple({"a": 1, "b": 2}, a, b), false, [c0#x, c1#x] + +- OneRowRelation + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'c') +-- !query analysis +Project [c0#x, c1#x] ++- Generate json_tuple({"a": 1, "b": 2}, a, c), false, [c0#x, c1#x] + +- OneRowRelation + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'a') +-- !query analysis +Project [c0#x, c1#x] ++- Generate json_tuple({"a": 1, "b": 2}, a, a), false, [c0#x, c1#x] + +- OneRowRelation + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x, y) +-- !query analysis +Project [x#x, y#x] ++- SubqueryAlias t + +- Project [c0#x AS x#x, c1#x AS y#x] + +- Generate json_tuple({"a": 1, "b": 2}, a, b), false, [c0#x, c1#x] + +- OneRowRelation + + +-- !query +select * from json_tuple('{"a": bad, "b": string}', 'a', 'b') +-- !query analysis +Project [c0#x, c1#x] ++- Generate json_tuple({"a": bad, "b": string}, a, b), false, [c0#x, c1#x] + +- OneRowRelation + + +-- !query +select * from json_tuple() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`json_tuple`" + } +} + + +-- !query +select * from json_tuple('{"a": 1}') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`json_tuple`" + } +} + + +-- !query +select * from json_tuple('{"a": 1}', 1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "funcName" : "`json_tuple`", + "sqlExpr" : "\"json_tuple({\"a\": 1}, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 39, + "fragment" : "json_tuple('{\"a\": 1}', 1)" + } ] +} + + +-- !query +select * from json_tuple('{"a": 1}', null) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "funcName" : "`json_tuple`", + "sqlExpr" : "\"json_tuple({\"a\": 1}, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 42, + "fragment" : "json_tuple('{\"a\": 1}', null)" + } ] +} + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2307", + "messageParameters" : { + "aliasesNum" : "1", + "funcName" : "json_tuple", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 62, + "fragment" : "json_tuple('{\"a\": 1, \"b\": 2}', 'a', 'b') AS t(x)" + } ] +} + + +-- !query +select * from stack(1, 1, 2, 3) +-- !query analysis +Project [col0#x, col1#x, col2#x] ++- Generate stack(1, 1, 2, 3), false, [col0#x, col1#x, col2#x] + +- OneRowRelation + + +-- !query +select * from stack(2, 1, 2, 3) +-- !query analysis +Project [col0#x, col1#x] ++- Generate stack(2, 1, 2, 3), false, [col0#x, col1#x] + +- OneRowRelation + + +-- !query +select * from stack(3, 1, 2, 3) t(x) +-- !query analysis +Project [x#x] ++- SubqueryAlias t + +- Project [col0#x AS x#x] + +- Generate stack(3, 1, 2, 3), false, [col0#x] + +- OneRowRelation + + +-- !query +select * from stack(4, 1, 2, 3) t(x) +-- !query analysis +Project [x#x] ++- SubqueryAlias t + +- Project [col0#x AS x#x] + +- Generate stack(4, 1, 2, 3), false, [col0#x] + +- OneRowRelation + + +-- !query +select * from stack(2, 1, 1.1, 'a', 2, 2.2, 'b') t(a, b, c) +-- !query analysis +Project [a#x, b#x, c#x] ++- SubqueryAlias t + +- Project [col0#x AS a#x, col1#x AS b#x, col2#x AS c#x] + +- Generate stack(2, 1, 1.1, a, 2, 2.2, b), false, [col0#x, col1#x, col2#x] + +- OneRowRelation + + +-- !query +select * from stack(2, 1, 1.1, null, 2, null, 'b') t(a, b, c) +-- !query analysis +Project [a#x, b#x, c#x] ++- SubqueryAlias t + +- Project [col0#x AS a#x, col1#x AS b#x, col2#x AS c#x] + +- Generate stack(2, 1, 1.1, null, 2, null, b), false, [col0#x, col1#x, col2#x] + +- OneRowRelation + + +-- !query +select * from stack() +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`stack`" + } +} + + +-- !query +select * from stack(2, 1, 2, 3) t(a, b, c) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2307", + "messageParameters" : { + "aliasesNum" : "3", + "funcName" : "stack", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 42, + "fragment" : "stack(2, 1, 2, 3) t(a, b, c)" + } ] +} + + +-- !query +select * from stack(2, 1, '1.1', 'a', 2, 2.2, 'b') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.STACK_COLUMN_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "columnIndex" : "1", + "leftParamIndex" : "2", + "leftType" : "\"STRING\"", + "rightParamIndex" : "5", + "rightType" : "\"DECIMAL(2,1)\"", + "sqlExpr" : "\"stack(2, 1, 1.1, a, 2, 2.2, b)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 50, + "fragment" : "stack(2, 1, '1.1', 'a', 2, 2.2, 'b')" + } ] +} + + +-- !query +select * from stack(2, explode(array(1, 2, 3))) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "0A000", + "messageParameters" : { + "expression" : "\"stack(2, explode(array(1, 2, 3)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 47, + "fragment" : "stack(2, explode(array(1, 2, 3)))" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/tablesample-negative.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/tablesample-negative.sql.out new file mode 100644 index 0000000000000..57ae6fb7dc7d1 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/tablesample-negative.sql.out @@ -0,0 +1,66 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE DATABASE mydb1 +-- !query analysis +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] + + +-- !query +USE mydb1 +-- !query analysis +SetCatalogAndNamespace ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] + + +-- !query +CREATE TABLE t1 USING parquet AS SELECT 1 AS i1 +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`mydb1`.`t1`, ErrorIfExists, [i1] + +- Project [1 AS i1#x] + +- OneRowRelation + + +-- !query +SELECT mydb1.t1 FROM t1 TABLESAMPLE (-1 PERCENT) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Sampling fraction (-0.01) must be on interval [0, 1]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 48, + "fragment" : "TABLESAMPLE (-1 PERCENT)" + } ] +} + + +-- !query +SELECT mydb1.t1 FROM t1 TABLESAMPLE (101 PERCENT) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Sampling fraction (1.01) must be on interval [0, 1]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 49, + "fragment" : "TABLESAMPLE (101 PERCENT)" + } ] +} + + +-- !query +DROP DATABASE mydb1 CASCADE +-- !query analysis +DropNamespace false, true ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [mydb1] 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 new file mode 100644 index 0000000000000..a965fac864849 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp-ltz.sql.out @@ -0,0 +1,53 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select timestamp_ltz'2016-12-31 00:12:00', timestamp_ltz'2016-12-31' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- 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)), 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)), None, TimestampType, Some(America/Los_Angeles), false) 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] ++- 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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +SELECT convert_timezone('Europe/Brussels', timestamp_ltz'2022-03-23 00:00:00 America/Los_Angeles') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] 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 new file mode 100644 index 0000000000000..bb5e4de56e085 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp-ntz.sql.out @@ -0,0 +1,139 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select timestamp_ntz'2016-12-31 00:12:00', timestamp_ntz'2016-12-31' +-- !query analysis +Project [2016-12-31 00:12:00 AS TIMESTAMP_NTZ '2016-12-31 00:12:00'#x, 2016-12-31 00:00:00 AS TIMESTAMP_NTZ '2016-12-31 00:00:00'#x] ++- OneRowRelation + + +-- !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] ++- 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)), 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)), None, TimestampNTZType, Some(America/Los_Angeles), false) 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] ++- 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] ++- OneRowRelation + + +-- !query +SELECT make_timestamp_ntz(2021, 07, 11, 6, 30, 45.678, 'CET') +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "7", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "6", + "functionName" : "`make_timestamp_ntz`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "make_timestamp_ntz(2021, 07, 11, 6, 30, 45.678, 'CET')" + } ] +} + + +-- !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] ++- OneRowRelation + + +-- !query +SELECT convert_timezone('Europe/Moscow', 'America/Los_Angeles', timestamp_ntz'2022-01-01 00:00:00') +-- !query analysis +Project [convert_timezone(Europe/Moscow, America/Los_Angeles, 2022-01-01 00:00:00) AS convert_timezone(Europe/Moscow, America/Los_Angeles, TIMESTAMP_NTZ '2022-01-01 00:00:00')#x] ++- OneRowRelation + + +-- !query +SELECT convert_timezone('Europe/Brussels', timestamp_ntz'2022-03-23 00:00:00') +-- !query analysis +Project [convert_timezone(current_timezone(), Europe/Brussels, 2022-03-23 00:00:00) AS convert_timezone(current_timezone(), Europe/Brussels, TIMESTAMP_NTZ '2022-03-23 00:00:00')#x] ++- OneRowRelation + + +-- !query +select timestampdiff(QUARTER, timestamp_ntz'2022-01-01 01:02:03', timestamp_ntz'2022-05-02 05:06:07') +-- !query analysis +Project [timestampdiff(QUARTER, cast(2022-01-01 01:02:03 as timestamp), cast(2022-05-02 05:06:07 as timestamp), Some(America/Los_Angeles)) AS timestampdiff(QUARTER, TIMESTAMP_NTZ '2022-01-01 01:02:03', TIMESTAMP_NTZ '2022-05-02 05:06:07')#xL] ++- OneRowRelation + + +-- !query +select timestampdiff(HOUR, timestamp_ntz'2022-02-14 01:02:03', timestamp_ltz'2022-02-14 02:03:04') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(YEAR, date'2022-02-15', timestamp_ntz'2023-02-15 10:11:12') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(MILLISECOND, timestamp_ntz'2022-02-14 23:59:59.123', date'2022-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp_ntz'2022-01-01 00:00:00' = date'2022-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp_ntz'2022-01-01 00:00:00' > date'2022-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp_ntz'2022-01-01 00:00:00' < date'2022-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp_ntz'2022-01-01 00:00:00' = timestamp_ltz'2022-01-01 00:00:00' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp_ntz'2022-01-01 00:00:00' > timestamp_ltz'2022-01-01 00:00:00' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp_ntz'2022-01-01 00:00:00' < timestamp_ltz'2022-01-01 00:00:00' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] 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 new file mode 100644 index 0000000000000..25e7cfcbcbb1f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp.sql.out @@ -0,0 +1,861 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select timestamp '2019-01-01\t' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp '2019-01-01中文' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2019-01-01中文'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} + + +-- !query +select timestamp'4294967297' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'4294967297'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} + + +-- !query +select timestamp'2021-01-01T12:30:4294967297.123456' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-01-01T12:30:4294967297.123456'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} + + +-- !query +select current_timestamp = current_timestamp +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select current_timestamp() = current_timestamp() +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select localtimestamp() = localtimestamp() +-- !query analysis +Project [(localtimestamp(Some(America/Los_Angeles)) = localtimestamp(Some(America/Los_Angeles))) AS (localtimestamp() = localtimestamp())#x] ++- OneRowRelation + + +-- !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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) +-- !query analysis +Project [timestamp_seconds(1230219000) AS timestamp_seconds(1230219000)#x, timestamp_seconds(-1230219000) AS timestamp_seconds(-1230219000)#x, timestamp_seconds(null) AS timestamp_seconds(NULL)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)) +-- !query analysis +Project [timestamp_seconds(1.23) AS timestamp_seconds(1.23)#x, timestamp_seconds(1.23) AS timestamp_seconds(1.23)#x, timestamp_seconds(cast(1.23 as float)) AS timestamp_seconds(1.23)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null) +-- !query analysis +Project [timestamp_millis(1230219000123) AS timestamp_millis(1230219000123)#x, timestamp_millis(-1230219000123) AS timestamp_millis(-1230219000123)#x, timestamp_millis(null) AS timestamp_millis(NULL)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null) +-- !query analysis +Project [timestamp_micros(1230219000123123) AS timestamp_micros(1230219000123123)#x, timestamp_micros(-1230219000123123) AS timestamp_micros(-1230219000123123)#x, timestamp_micros(null) AS timestamp_micros(NULL)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(1230219000123123) +-- !query analysis +Project [timestamp_seconds(1230219000123123) AS timestamp_seconds(1230219000123123)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(-1230219000123123) +-- !query analysis +Project [timestamp_seconds(-1230219000123123) AS timestamp_seconds(-1230219000123123)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MILLIS(92233720368547758) +-- !query analysis +Project [timestamp_millis(92233720368547758) AS timestamp_millis(92233720368547758)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MILLIS(-92233720368547758) +-- !query analysis +Project [timestamp_millis(-92233720368547758) AS timestamp_millis(-92233720368547758)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(0.1234567) +-- !query analysis +Project [timestamp_seconds(0.1234567) AS timestamp_seconds(0.1234567)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)) +-- !query analysis +Project [timestamp_seconds(0.1234567) AS timestamp_seconds(0.1234567)#x, timestamp_seconds(cast(0.1234567 as float)) AS timestamp_seconds(0.1234567)#x] ++- OneRowRelation + + +-- !query +create temporary view ttf1 as select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`) +-- !query analysis +CreateViewCommand `ttf1`, select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`), false, false, LocalTempView, true + +- Project [current_date#x, current_timestamp#x] + +- SubqueryAlias ttf1 + +- LocalRelation [current_date#x, current_timestamp#x] + + +-- !query +select typeof(current_date), typeof(current_timestamp) from ttf1 +-- !query analysis +Project [typeof(current_date#x) AS typeof(current_date)#x, typeof(current_timestamp#x) AS typeof(current_timestamp)#x] ++- SubqueryAlias ttf1 + +- View (`ttf1`, [current_date#x,current_timestamp#x]) + +- Project [cast(current_date#x as int) AS current_date#x, cast(current_timestamp#x as int) AS current_timestamp#x] + +- Project [current_date#x, current_timestamp#x] + +- SubqueryAlias ttf1 + +- LocalRelation [current_date#x, current_timestamp#x] + + +-- !query +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b) +-- !query analysis +CreateViewCommand `ttf2`, select * from values + (1, 2), + (2, 3) + as ttf2(a, b), false, false, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias ttf2 + +- LocalRelation [a#x, b#x] + + +-- !query +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2 +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select a, b from ttf2 order by a, current_date +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- OneRowRelation + + +-- !query +select to_timestamp(1) +-- !query analysis +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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), 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), false) AS to_timestamp(2019 10:10:10, yyyy hh:mm:ss)#x] ++- OneRowRelation + + +-- !query +select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:10\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "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 +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:11\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "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 +select timestamp'2011-11-11 11:11:11' - null +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select null - timestamp'2011-11-11 11:11:11' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +create temporary view ts_view as select '2011-11-11 11:11:11' str +-- !query analysis +CreateViewCommand `ts_view`, select '2011-11-11 11:11:11' str, false, false, LocalTempView, true + +- Project [2011-11-11 11:11:11 AS str#x] + +- OneRowRelation + + +-- !query +select str - timestamp'2011-11-11 11:11:11' from ts_view +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "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 +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "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 +select timestamp'2011-11-11 11:11:11' + '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" + } ] +} + + +-- !query +select '1' + timestamp'2011-11-11 11:11:11' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + null +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "timestamp'2011-11-11 11:11:11' + null" + } ] +} + + +-- !query +select null + timestamp'2011-11-11 11:11:11' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "null + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + interval '2' day, + timestamp'2011-11-11 11:11:11' - interval '2-2' year to month, + timestamp'2011-11-11 11:11:11' + interval '-2' second, + timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second, + - interval 2 years + timestamp'2011-11-11 11:11:11', + interval '1 12' day to hour + timestamp'2011-11-11 11:11:11' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select date '2012-01-01' - interval 3 hours, + date '2012-01-01' + interval '12:12:12' hour to second, + interval '2' minute + date '2012-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- 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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_json(StructField(t,TimestampType,true), (timestampFormat,dd/MMMMM/yyyy), {"t":"26/October/2015"}, Some(America/Los_Angeles)) AS from_json({"t":"26/October/2015"})#x] ++- OneRowRelation + + +-- !query +select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_csv(StructField(t,TimestampType,true), (timestampFormat,dd/MMMMM/yyyy), 26/October/2015, Some(America/Los_Angeles), None) AS from_csv(26/October/2015)#x] ++- OneRowRelation + + +-- !query +select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampadd(YEAR, 1, date'2022-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampadd(SECOND, -1, date'2022-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/datetime-special.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/datetime-special.sql.out new file mode 100644 index 0000000000000..7a7bd07ab61b6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/datetime-special.sql.out @@ -0,0 +1,26 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select date'999999-03-18', date'-0001-1-28', date'0015' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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] ++- OneRowRelation + + +-- !query +select timestamp'-1969-12-31 16:00:00', timestamp'-0015-03-18 16:00:00', timestamp'-000001', timestamp'99999-03-18T12:03:17' +-- !query analysis +Project [-1969-12-31 16:00:00 AS TIMESTAMP_NTZ '-1969-12-31 16:00:00'#x, -0015-03-18 16:00:00 AS TIMESTAMP_NTZ '-0015-03-18 16:00:00'#x, -0001-01-01 00:00:00 AS TIMESTAMP_NTZ '-0001-01-01 00:00:00'#x, +99999-03-18 12:03:17 AS TIMESTAMP_NTZ '+99999-03-18 12:03:17'#x] ++- OneRowRelation + + +-- !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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) AS make_timestamp(99999, 3, 18, 12, 3, 17.0)#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp-ansi.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp-ansi.sql.out new file mode 100644 index 0000000000000..630e49bfaa506 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp-ansi.sql.out @@ -0,0 +1,808 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select timestamp '2019-01-01\t' +-- !query analysis +Project [2019-01-01 00:00:00 AS TIMESTAMP_NTZ '2019-01-01 00:00:00'#x] ++- OneRowRelation + + +-- !query +select timestamp '2019-01-01中文' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2019-01-01中文'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} + + +-- !query +select timestamp'4294967297' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'4294967297'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} + + +-- !query +select timestamp'2021-01-01T12:30:4294967297.123456' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-01-01T12:30:4294967297.123456'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} + + +-- !query +select current_timestamp = current_timestamp +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select current_timestamp() = current_timestamp() +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select localtimestamp() = localtimestamp() +-- !query analysis +Project [(localtimestamp(Some(America/Los_Angeles)) = localtimestamp(Some(America/Los_Angeles))) AS (localtimestamp() = localtimestamp())#x] ++- OneRowRelation + + +-- !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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) AS make_timestamp(1, 1, 1, 1, 1, 999.999999)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) +-- !query analysis +Project [timestamp_seconds(1230219000) AS timestamp_seconds(1230219000)#x, timestamp_seconds(-1230219000) AS timestamp_seconds(-1230219000)#x, timestamp_seconds(null) AS timestamp_seconds(NULL)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)) +-- !query analysis +Project [timestamp_seconds(1.23) AS timestamp_seconds(1.23)#x, timestamp_seconds(1.23) AS timestamp_seconds(1.23)#x, timestamp_seconds(cast(1.23 as float)) AS timestamp_seconds(1.23)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null) +-- !query analysis +Project [timestamp_millis(1230219000123) AS timestamp_millis(1230219000123)#x, timestamp_millis(-1230219000123) AS timestamp_millis(-1230219000123)#x, timestamp_millis(null) AS timestamp_millis(NULL)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null) +-- !query analysis +Project [timestamp_micros(1230219000123123) AS timestamp_micros(1230219000123123)#x, timestamp_micros(-1230219000123123) AS timestamp_micros(-1230219000123123)#x, timestamp_micros(null) AS timestamp_micros(NULL)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(1230219000123123) +-- !query analysis +Project [timestamp_seconds(1230219000123123) AS timestamp_seconds(1230219000123123)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(-1230219000123123) +-- !query analysis +Project [timestamp_seconds(-1230219000123123) AS timestamp_seconds(-1230219000123123)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MILLIS(92233720368547758) +-- !query analysis +Project [timestamp_millis(92233720368547758) AS timestamp_millis(92233720368547758)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MILLIS(-92233720368547758) +-- !query analysis +Project [timestamp_millis(-92233720368547758) AS timestamp_millis(-92233720368547758)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(0.1234567) +-- !query analysis +Project [timestamp_seconds(0.1234567) AS timestamp_seconds(0.1234567)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)) +-- !query analysis +Project [timestamp_seconds(0.1234567) AS timestamp_seconds(0.1234567)#x, timestamp_seconds(cast(0.1234567 as float)) AS timestamp_seconds(0.1234567)#x] ++- OneRowRelation + + +-- !query +create temporary view ttf1 as select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`) +-- !query analysis +CreateViewCommand `ttf1`, select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`), false, false, LocalTempView, true + +- Project [current_date#x, current_timestamp#x] + +- SubqueryAlias ttf1 + +- LocalRelation [current_date#x, current_timestamp#x] + + +-- !query +select typeof(current_date), typeof(current_timestamp) from ttf1 +-- !query analysis +Project [typeof(current_date#x) AS typeof(current_date)#x, typeof(current_timestamp#x) AS typeof(current_timestamp)#x] ++- SubqueryAlias ttf1 + +- View (`ttf1`, [current_date#x,current_timestamp#x]) + +- Project [cast(current_date#x as int) AS current_date#x, cast(current_timestamp#x as int) AS current_timestamp#x] + +- Project [current_date#x, current_timestamp#x] + +- SubqueryAlias ttf1 + +- LocalRelation [current_date#x, current_timestamp#x] + + +-- !query +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b) +-- !query analysis +CreateViewCommand `ttf2`, select * from values + (1, 2), + (2, 3) + as ttf2(a, b), false, false, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias ttf2 + +- LocalRelation [a#x, b#x] + + +-- !query +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2 +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select a, b from ttf2 order by a, current_date +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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, TimestampNTZType, Some(America/Los_Angeles), true) AS to_timestamp(NULL)#x, to_timestamp(2016-12-31 00:12:00, None, TimestampNTZType, Some(America/Los_Angeles), true) AS to_timestamp(2016-12-31 00:12:00)#x, to_timestamp(2016-12-31, Some(yyyy-MM-dd), TimestampNTZType, 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(cast(1 as string), None, TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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'), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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''), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, Some(America/Los_Angeles), true) AS to_timestamp(2019 10:10:10, yyyy hh:mm:ss)#x] ++- OneRowRelation + + +-- !query +select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query analysis +Project [(2011-11-11 11:11:11 - 2011-11-11 11:11:10) AS (TIMESTAMP_NTZ '2011-11-11 11:11:11' - TIMESTAMP_NTZ '2011-11-11 11:11:10')#x] ++- OneRowRelation + + +-- !query +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' +-- !query analysis +Project [(2011-11-11 11:11:11 - cast(2011-11-11 11:11:10 as timestamp_ntz)) AS (TIMESTAMP_NTZ '2011-11-11 11:11:11' - 2011-11-11 11:11:10)#x] ++- OneRowRelation + + +-- !query +select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query analysis +Project [(cast(2011-11-11 11:11:11 as timestamp_ntz) - 2011-11-11 11:11:10) AS (2011-11-11 11:11:11 - TIMESTAMP_NTZ '2011-11-11 11:11:10')#x] ++- OneRowRelation + + +-- !query +select timestamp'2011-11-11 11:11:11' - null +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select null - timestamp'2011-11-11 11:11:11' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +create temporary view ts_view as select '2011-11-11 11:11:11' str +-- !query analysis +CreateViewCommand `ts_view`, select '2011-11-11 11:11:11' str, false, false, LocalTempView, true + +- Project [2011-11-11 11:11:11 AS str#x] + +- OneRowRelation + + +-- !query +select str - timestamp'2011-11-11 11:11:11' from ts_view +-- !query analysis +Project [(cast(str#x as timestamp_ntz) - 2011-11-11 11:11:11) AS (str - TIMESTAMP_NTZ '2011-11-11 11:11:11')#x] ++- SubqueryAlias ts_view + +- View (`ts_view`, [str#x]) + +- Project [cast(str#x as string) AS str#x] + +- Project [2011-11-11 11:11:11 AS str#x] + +- OneRowRelation + + +-- !query +select timestamp'2011-11-11 11:11:11' - str from ts_view +-- !query analysis +Project [(2011-11-11 11:11:11 - cast(str#x as timestamp_ntz)) AS (TIMESTAMP_NTZ '2011-11-11 11:11:11' - str)#x] ++- SubqueryAlias ts_view + +- View (`ts_view`, [str#x]) + +- Project [cast(str#x as string) AS str#x] + +- Project [2011-11-11 11:11:11 AS str#x] + +- OneRowRelation + + +-- !query +select timestamp'2011-11-11 11:11:11' + '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP_NTZ\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" + } ] +} + + +-- !query +select '1' + timestamp'2011-11-11 11:11:11' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP_NTZ\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(1 + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + null +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP_NTZ\"", + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "timestamp'2011-11-11 11:11:11' + null" + } ] +} + + +-- !query +select null + timestamp'2011-11-11 11:11:11' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"TIMESTAMP_NTZ\"", + "sqlExpr" : "\"(NULL + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "null + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + interval '2' day, + timestamp'2011-11-11 11:11:11' - interval '2-2' year to month, + timestamp'2011-11-11 11:11:11' + interval '-2' second, + timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second, + - interval 2 years + timestamp'2011-11-11 11:11:11', + interval '1 12' day to hour + timestamp'2011-11-11 11:11:11' +-- !query analysis +Project [cast(2011-11-11 11:11:11 + INTERVAL '2' DAY as timestamp_ntz) AS TIMESTAMP_NTZ '2011-11-11 11:11:11' + INTERVAL '2' DAY#x, 2011-11-11 11:11:11 - INTERVAL '2-2' YEAR TO MONTH AS TIMESTAMP_NTZ '2011-11-11 11:11:11' - INTERVAL '2-2' YEAR TO MONTH#x, cast(2011-11-11 11:11:11 + INTERVAL '-02' SECOND as timestamp_ntz) AS TIMESTAMP_NTZ '2011-11-11 11:11:11' + INTERVAL '-02' SECOND#x, cast(2011-11-11 11:11:11 - INTERVAL '12:12:12.123456' HOUR TO SECOND as timestamp_ntz) AS TIMESTAMP_NTZ '2011-11-11 11:11:11' - INTERVAL '12:12:12.123456' HOUR TO SECOND#x, 2011-11-11 11:11:11 + -INTERVAL '2' YEAR AS TIMESTAMP_NTZ '2011-11-11 11:11:11' + (- INTERVAL '2' YEAR)#x, cast(2011-11-11 11:11:11 + INTERVAL '1 12' DAY TO HOUR as timestamp_ntz) AS TIMESTAMP_NTZ '2011-11-11 11:11:11' + INTERVAL '1 12' DAY TO HOUR#x] ++- OneRowRelation + + +-- !query +select date '2012-01-01' - interval 3 hours, + date '2012-01-01' + interval '12:12:12' hour to second, + interval '2' minute + date '2012-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), true) AS unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#xL] ++- OneRowRelation + + +-- !query +select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_json(StructField(t,TimestampNTZType,true), (timestampFormat,dd/MMMMM/yyyy), {"t":"26/October/2015"}, Some(America/Los_Angeles)) AS from_json({"t":"26/October/2015"})#x] ++- OneRowRelation + + +-- !query +select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_csv(StructField(t,TimestampNTZType,true), (timestampFormat,dd/MMMMM/yyyy), 26/October/2015, Some(America/Los_Angeles), None) AS from_csv(26/October/2015)#x] ++- OneRowRelation + + +-- !query +select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03') +-- !query analysis +Project [timestampadd(MONTH, -1, 2022-02-14 01:02:03, Some(America/Los_Angeles)) AS timestampadd(MONTH, -1, TIMESTAMP_NTZ '2022-02-14 01:02:03')#x] ++- OneRowRelation + + +-- !query +select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03') +-- !query analysis +Project [timestampadd(MINUTE, 58, 2022-02-14 01:02:03, Some(America/Los_Angeles)) AS timestampadd(MINUTE, 58, TIMESTAMP_NTZ '2022-02-14 01:02:03')#x] ++- OneRowRelation + + +-- !query +select timestampadd(YEAR, 1, date'2022-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampadd(SECOND, -1, date'2022-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query analysis +Project [timestampdiff(MONTH, cast(2022-02-14 01:02:03 as timestamp), cast(2022-01-14 01:02:03 as timestamp), Some(America/Los_Angeles)) AS timestampdiff(MONTH, TIMESTAMP_NTZ '2022-02-14 01:02:03', TIMESTAMP_NTZ '2022-01-14 01:02:03')#xL] ++- OneRowRelation + + +-- !query +select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query analysis +Project [timestampdiff(MINUTE, cast(2022-02-14 01:02:03 as timestamp), cast(2022-02-14 02:00:03 as timestamp), Some(America/Los_Angeles)) AS timestampdiff(MINUTE, TIMESTAMP_NTZ '2022-02-14 01:02:03', TIMESTAMP_NTZ '2022-02-14 02:00:03')#xL] ++- OneRowRelation + + +-- !query +select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp.sql.out new file mode 100644 index 0000000000000..d6e29e72682c6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp.sql.out @@ -0,0 +1,868 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select timestamp '2019-01-01\t' +-- !query analysis +Project [2019-01-01 00:00:00 AS TIMESTAMP_NTZ '2019-01-01 00:00:00'#x] ++- OneRowRelation + + +-- !query +select timestamp '2019-01-01中文' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2019-01-01中文'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} + + +-- !query +select timestamp'4294967297' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'4294967297'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} + + +-- !query +select timestamp'2021-01-01T12:30:4294967297.123456' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-01-01T12:30:4294967297.123456'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} + + +-- !query +select current_timestamp = current_timestamp +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select current_timestamp() = current_timestamp() +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select localtimestamp() = localtimestamp() +-- !query analysis +Project [(localtimestamp(Some(America/Los_Angeles)) = localtimestamp(Some(America/Los_Angeles))) AS (localtimestamp() = localtimestamp())#x] ++- OneRowRelation + + +-- !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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) 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, TimestampNTZType) AS make_timestamp(1, 1, 1, 1, 1, 999.999999)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) +-- !query analysis +Project [timestamp_seconds(1230219000) AS timestamp_seconds(1230219000)#x, timestamp_seconds(-1230219000) AS timestamp_seconds(-1230219000)#x, timestamp_seconds(null) AS timestamp_seconds(NULL)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)) +-- !query analysis +Project [timestamp_seconds(1.23) AS timestamp_seconds(1.23)#x, timestamp_seconds(1.23) AS timestamp_seconds(1.23)#x, timestamp_seconds(cast(1.23 as float)) AS timestamp_seconds(1.23)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null) +-- !query analysis +Project [timestamp_millis(1230219000123) AS timestamp_millis(1230219000123)#x, timestamp_millis(-1230219000123) AS timestamp_millis(-1230219000123)#x, timestamp_millis(null) AS timestamp_millis(NULL)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null) +-- !query analysis +Project [timestamp_micros(1230219000123123) AS timestamp_micros(1230219000123123)#x, timestamp_micros(-1230219000123123) AS timestamp_micros(-1230219000123123)#x, timestamp_micros(null) AS timestamp_micros(NULL)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(1230219000123123) +-- !query analysis +Project [timestamp_seconds(1230219000123123) AS timestamp_seconds(1230219000123123)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(-1230219000123123) +-- !query analysis +Project [timestamp_seconds(-1230219000123123) AS timestamp_seconds(-1230219000123123)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MILLIS(92233720368547758) +-- !query analysis +Project [timestamp_millis(92233720368547758) AS timestamp_millis(92233720368547758)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_MILLIS(-92233720368547758) +-- !query analysis +Project [timestamp_millis(-92233720368547758) AS timestamp_millis(-92233720368547758)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(0.1234567) +-- !query analysis +Project [timestamp_seconds(0.1234567) AS timestamp_seconds(0.1234567)#x] ++- OneRowRelation + + +-- !query +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)) +-- !query analysis +Project [timestamp_seconds(0.1234567) AS timestamp_seconds(0.1234567)#x, timestamp_seconds(cast(0.1234567 as float)) AS timestamp_seconds(0.1234567)#x] ++- OneRowRelation + + +-- !query +create temporary view ttf1 as select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`) +-- !query analysis +CreateViewCommand `ttf1`, select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`), false, false, LocalTempView, true + +- Project [current_date#x, current_timestamp#x] + +- SubqueryAlias ttf1 + +- LocalRelation [current_date#x, current_timestamp#x] + + +-- !query +select typeof(current_date), typeof(current_timestamp) from ttf1 +-- !query analysis +Project [typeof(current_date#x) AS typeof(current_date)#x, typeof(current_timestamp#x) AS typeof(current_timestamp)#x] ++- SubqueryAlias ttf1 + +- View (`ttf1`, [current_date#x,current_timestamp#x]) + +- Project [cast(current_date#x as int) AS current_date#x, cast(current_timestamp#x as int) AS current_timestamp#x] + +- Project [current_date#x, current_timestamp#x] + +- SubqueryAlias ttf1 + +- LocalRelation [current_date#x, current_timestamp#x] + + +-- !query +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b) +-- !query analysis +CreateViewCommand `ttf2`, select * from values + (1, 2), + (2, 3) + as ttf2(a, b), false, false, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias ttf2 + +- LocalRelation [a#x, b#x] + + +-- !query +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2 +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select a, b from ttf2 order by a, current_date +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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, TimestampNTZType, Some(America/Los_Angeles), false) AS to_timestamp(NULL)#x, to_timestamp(2016-12-31 00:12:00, None, TimestampNTZType, Some(America/Los_Angeles), false) AS to_timestamp(2016-12-31 00:12:00)#x, to_timestamp(2016-12-31, Some(yyyy-MM-dd), TimestampNTZType, 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(cast(1 as string), None, TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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]), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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'), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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''), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, Some(America/Los_Angeles), false) AS to_timestamp(2019 10:10:10, yyyy hh:mm:ss)#x] ++- OneRowRelation + + +-- !query +select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query analysis +Project [(2011-11-11 11:11:11 - 2011-11-11 11:11:10) AS (TIMESTAMP_NTZ '2011-11-11 11:11:11' - TIMESTAMP_NTZ '2011-11-11 11:11:10')#x] ++- OneRowRelation + + +-- !query +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:10\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(TIMESTAMP_NTZ '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 +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:11\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(2011-11-11 11:11:11 - TIMESTAMP_NTZ '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 +select timestamp'2011-11-11 11:11:11' - null +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select null - timestamp'2011-11-11 11:11:11' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +create temporary view ts_view as select '2011-11-11 11:11:11' str +-- !query analysis +CreateViewCommand `ts_view`, select '2011-11-11 11:11:11' str, false, false, LocalTempView, true + +- Project [2011-11-11 11:11:11 AS str#x] + +- OneRowRelation + + +-- !query +select str - timestamp'2011-11-11 11:11:11' from ts_view +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(str - TIMESTAMP_NTZ '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 +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' - str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' - str" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + '1' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP_NTZ\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" + } ] +} + + +-- !query +select '1' + timestamp'2011-11-11 11:11:11' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP_NTZ\"", + "sqlExpr" : "\"(1 + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + null +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP_NTZ\"", + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "timestamp'2011-11-11 11:11:11' + null" + } ] +} + + +-- !query +select null + timestamp'2011-11-11 11:11:11' +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"TIMESTAMP_NTZ\"", + "sqlExpr" : "\"(NULL + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "null + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + interval '2' day, + timestamp'2011-11-11 11:11:11' - interval '2-2' year to month, + timestamp'2011-11-11 11:11:11' + interval '-2' second, + timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second, + - interval 2 years + timestamp'2011-11-11 11:11:11', + interval '1 12' day to hour + timestamp'2011-11-11 11:11:11' +-- !query analysis +Project [cast(2011-11-11 11:11:11 + INTERVAL '2' DAY as timestamp_ntz) AS TIMESTAMP_NTZ '2011-11-11 11:11:11' + INTERVAL '2' DAY#x, 2011-11-11 11:11:11 - INTERVAL '2-2' YEAR TO MONTH AS TIMESTAMP_NTZ '2011-11-11 11:11:11' - INTERVAL '2-2' YEAR TO MONTH#x, cast(2011-11-11 11:11:11 + INTERVAL '-02' SECOND as timestamp_ntz) AS TIMESTAMP_NTZ '2011-11-11 11:11:11' + INTERVAL '-02' SECOND#x, cast(2011-11-11 11:11:11 - INTERVAL '12:12:12.123456' HOUR TO SECOND as timestamp_ntz) AS TIMESTAMP_NTZ '2011-11-11 11:11:11' - INTERVAL '12:12:12.123456' HOUR TO SECOND#x, 2011-11-11 11:11:11 + -INTERVAL '2' YEAR AS TIMESTAMP_NTZ '2011-11-11 11:11:11' + (- INTERVAL '2' YEAR)#x, cast(2011-11-11 11:11:11 + INTERVAL '1 12' DAY TO HOUR as timestamp_ntz) AS TIMESTAMP_NTZ '2011-11-11 11:11:11' + INTERVAL '1 12' DAY TO HOUR#x] ++- OneRowRelation + + +-- !query +select date '2012-01-01' - interval 3 hours, + date '2012-01-01' + interval '12:12:12' hour to second, + interval '2' minute + date '2012-01-01' +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !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), TimestampNTZType, 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), TimestampNTZType, 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), TimestampNTZType, 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), false) AS unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#xL] ++- OneRowRelation + + +-- !query +select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_json(StructField(t,TimestampNTZType,true), (timestampFormat,dd/MMMMM/yyyy), {"t":"26/October/2015"}, Some(America/Los_Angeles)) AS from_json({"t":"26/October/2015"})#x] ++- OneRowRelation + + +-- !query +select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query analysis +Project [from_csv(StructField(t,TimestampNTZType,true), (timestampFormat,dd/MMMMM/yyyy), 26/October/2015, Some(America/Los_Angeles), None) AS from_csv(26/October/2015)#x] ++- OneRowRelation + + +-- !query +select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03') +-- !query analysis +Project [timestampadd(MONTH, -1, 2022-02-14 01:02:03, Some(America/Los_Angeles)) AS timestampadd(MONTH, -1, TIMESTAMP_NTZ '2022-02-14 01:02:03')#x] ++- OneRowRelation + + +-- !query +select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03') +-- !query analysis +Project [timestampadd(MINUTE, 58, 2022-02-14 01:02:03, Some(America/Los_Angeles)) AS timestampadd(MINUTE, 58, TIMESTAMP_NTZ '2022-02-14 01:02:03')#x] ++- OneRowRelation + + +-- !query +select timestampadd(YEAR, 1, date'2022-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampadd(SECOND, -1, date'2022-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query analysis +Project [timestampdiff(MONTH, cast(2022-02-14 01:02:03 as timestamp), cast(2022-01-14 01:02:03 as timestamp), Some(America/Los_Angeles)) AS timestampdiff(MONTH, TIMESTAMP_NTZ '2022-02-14 01:02:03', TIMESTAMP_NTZ '2022-01-14 01:02:03')#xL] ++- OneRowRelation + + +-- !query +select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query analysis +Project [timestampdiff(MINUTE, cast(2022-02-14 01:02:03 as timestamp), cast(2022-02-14 02:00:03 as timestamp), Some(America/Los_Angeles)) AS timestampdiff(MINUTE, TIMESTAMP_NTZ '2022-02-14 01:02:03', TIMESTAMP_NTZ '2022-02-14 02:00:03')#xL] ++- OneRowRelation + + +-- !query +select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timezone.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timezone.sql.out new file mode 100644 index 0000000000000..2ffbb963582ac --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timezone.sql.out @@ -0,0 +1,132 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SET TIME ZONE 'Asia/Hong_Kong' +-- !query analysis +SetCommand (spark.sql.session.timeZone,Some(Asia/Hong_Kong)) + + +-- !query +SET TIME ZONE 'GMT+1' +-- !query analysis +SetCommand (spark.sql.session.timeZone,Some(GMT+1)) + + +-- !query +SET TIME ZONE INTERVAL 10 HOURS +-- !query analysis +SetCommand (spark.sql.session.timeZone,Some(+10:00)) + + +-- !query +SET TIME ZONE INTERVAL '15:40:32' HOUR TO SECOND +-- !query analysis +SetCommand (spark.sql.session.timeZone,Some(+15:40:32)) + + +-- !query +SET TIME ZONE LOCAL +-- !query analysis +SetCommand (spark.sql.session.timeZone,Some(America/Los_Angeles)) + + +-- !query +SET TIME ZONE +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0045", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 13, + "fragment" : "SET TIME ZONE" + } ] +} + + +-- !query +SET TIME ZONE 'invalid/zone' +-- !query analysis +java.lang.IllegalArgumentException +'invalid/zone' in spark.sql.session.timeZone is invalid. Cannot resolve the given timezone with ZoneId.of(_, ZoneId.SHORT_IDS) + + +-- !query +SET TIME ZONE INTERVAL 3 DAYS +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "SET TIME ZONE INTERVAL 3 DAYS" + } ] +} + + +-- !query +SET TIME ZONE INTERVAL 24 HOURS +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 31, + "fragment" : "SET TIME ZONE INTERVAL 24 HOURS" + } ] +} + + +-- !query +SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 48, + "fragment" : "SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND" + } ] +} + + +-- !query +SET TIME ZONE INTERVAL 10 HOURS 'GMT+1' +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0045", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 39, + "fragment" : "SET TIME ZONE INTERVAL 10 HOURS 'GMT+1'" + } ] +} + + +-- !query +SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 45, + "fragment" : "SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND" + } ] +} 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 new file mode 100644 index 0000000000000..cda76f716a8a8 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out @@ -0,0 +1,1037 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW t AS SELECT * FROM VALUES +('1', true, unhex('537061726B2053514C'), tinyint(1), 1, smallint(100), bigint(1), float(1.0), 1.0, Decimal(1.0), timestamp('1997-01-02'), date('2000-04-01')), +('2', false, unhex('537061726B2053514C'), tinyint(2), 2, smallint(200), bigint(2), float(2.0), 2.0, Decimal(2.0), timestamp('1997-01-02 03:04:05'), date('2000-04-02')), +('3', true, unhex('537061726B2053514C'), tinyint(3), 3, smallint(300), bigint(3), float(3.0), 3.0, Decimal(3.0), timestamp('1997-02-10 17:32:01-08'), date('2000-04-03')) +AS t(a, b, c, d, e, f, g, h, i, j, k, l) +-- !query analysis +CreateViewCommand `t`, SELECT * FROM VALUES +('1', true, unhex('537061726B2053514C'), tinyint(1), 1, smallint(100), bigint(1), float(1.0), 1.0, Decimal(1.0), timestamp('1997-01-02'), date('2000-04-01')), +('2', false, unhex('537061726B2053514C'), tinyint(2), 2, smallint(200), bigint(2), float(2.0), 2.0, Decimal(2.0), timestamp('1997-01-02 03:04:05'), date('2000-04-02')), +('3', true, unhex('537061726B2053514C'), tinyint(3), 3, smallint(300), bigint(3), float(3.0), 3.0, Decimal(3.0), timestamp('1997-02-10 17:32:01-08'), date('2000-04-03')) +AS t(a, b, c, d, e, f, g, h, i, j, k, l), false, true, LocalTempView, true + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW script_trans AS SELECT * FROM VALUES +(1, 2, 3), +(4, 5, 6), +(7, 8, 9) +AS script_trans(a, b, c) +-- !query analysis +CreateViewCommand `script_trans`, SELECT * FROM VALUES +(1, 2, 3), +(4, 5, 6), +(7, 8, 9) +AS script_trans(a, b, c), false, true, LocalTempView, true + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW complex_trans AS SELECT * FROM VALUES +(1, 1), +(1, 1), +(2, 2), +(2, 2), +(3, 3), +(2, 2), +(3, 3), +(1, 1), +(3, 3) +as complex_trans(a, b) +-- !query analysis +CreateViewCommand `complex_trans`, SELECT * FROM VALUES +(1, 1), +(1, 1), +(2, 2), +(2, 2), +(3, 3), +(2, 2), +(3, 3), +(1, 1), +(3, 3) +as complex_trans(a, b), false, true, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias complex_trans + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT TRANSFORM(a) +USING 'cat' AS (a) +FROM t +-- !query analysis +ScriptTransformation cat, [a#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Project [a#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + USING 'cat' AS ( + a string, + b boolean, + c binary, + d tinyint, + e int, + f smallint, + g long, + h float, + i double, + j decimal(38, 18), + k timestamp, + l date) + FROM t +) tmp +-- !query analysis +Project [a#x, b#x, decode(c#x, UTF-8) AS decode(c, UTF-8)#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] ++- SubqueryAlias tmp + +- ScriptTransformation cat, [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + USING 'cat' AS ( + a string, + b string, + c string, + d string, + e string, + f string, + g string, + h string, + i string, + j string, + k string, + l string) + FROM t +) tmp +-- !query analysis +Project [a#x, b#x, decode(c#x, UTF-8) AS decode(c, UTF-8)#x, d#x, e#x, f#x, g#x, h#x, i#x, j#x, k#x, l#x] ++- SubqueryAlias tmp + +- ScriptTransformation cat, [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, i#x, j#x, k#x, l#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +SELECT TRANSFORM(a) +USING 'cat' +FROM t +-- !query analysis +ScriptTransformation cat, [key#x, value#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,true) ++- Project [a#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +SELECT TRANSFORM(a, b) +USING 'cat' +FROM t +-- !query analysis +ScriptTransformation cat, [key#x, value#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,true) ++- Project [a#x, b#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +SELECT TRANSFORM(a, b, c) +USING 'cat' +FROM t +-- !query analysis +ScriptTransformation cat, [key#x, value#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,true) ++- Project [a#x, b#x, c#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +SELECT TRANSFORM(a, b, c, d, e, f, g, h, i) +USING 'cat' AS (a int, b short, c long, d byte, e float, f double, g decimal(38, 18), h date, i timestamp) +FROM VALUES +('a','','1231a','a','213.21a','213.21a','0a.21d','2000-04-01123','1997-0102 00:00:') tmp(a, b, c, d, e, f, g, h, i) +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#xL, d#x, e#x, f#x, g#x, h#x, i#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, i#x] + +- SubqueryAlias tmp + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, i#x] + + +-- !query +SELECT TRANSFORM(b, max(a), sum(f)) +USING 'cat' AS (a, b) +FROM t +GROUP BY b +-- !query analysis +ScriptTransformation cat, [a#x, b#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Aggregate [b#x], [b#x, max(a#x) AS max(a)#x, sum(f#x) AS sum(f)#xL] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +MAP a, b USING 'cat' AS (a, b) FROM t +-- !query analysis +ScriptTransformation cat, [a#x, b#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Project [a#x, b#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +REDUCE a, b USING 'cat' AS (a, b) FROM t +-- !query analysis +ScriptTransformation cat, [a#x, b#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Project [a#x, b#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +USING 'cat' AS (a, b, c, d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +FROM t +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x, d#x], ScriptInputOutputSchema(List((TOK_TABLEROWFORMATFIELD,@), (TOK_TABLEROWFORMATNULL,NULL), (TOK_TABLEROWFORMATLINES, +)),List((TOK_TABLEROWFORMATFIELD,@), (TOK_TABLEROWFORMATNULL,NULL), (TOK_TABLEROWFORMATLINES, +)),None,None,List(),List(),None,None,false) ++- Project [a#x, b#x, c#x, null AS NULL#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +USING 'cat' AS (d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +FROM t +-- !query analysis +ScriptTransformation cat, [d#x], ScriptInputOutputSchema(List((TOK_TABLEROWFORMATFIELD,@), (TOK_TABLEROWFORMATNULL,NULL), (TOK_TABLEROWFORMATLINES, +)),List((TOK_TABLEROWFORMATFIELD,@), (TOK_TABLEROWFORMATNULL,NULL), (TOK_TABLEROWFORMATLINES, +)),None,None,List(),List(),None,None,false) ++- Project [a#x, b#x, c#x, null AS NULL#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' +USING 'cat' AS (a, b, c, d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +FROM t +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x, d#x], ScriptInputOutputSchema(List((TOK_TABLEROWFORMATFIELD,@), (TOK_TABLEROWFORMATLINES, +)),List((TOK_TABLEROWFORMATFIELD,@), (TOK_TABLEROWFORMATNULL,NULL), (TOK_TABLEROWFORMATLINES, +)),None,None,List(),List(),None,None,false) ++- Project [a#x, b#x, c#x, null AS NULL#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' +USING 'cat' AS (a, b, c, d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' +FROM t +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x, d#x], ScriptInputOutputSchema(List((TOK_TABLEROWFORMATFIELD,@), (TOK_TABLEROWFORMATLINES, +)),List((TOK_TABLEROWFORMATFIELD,@), (TOK_TABLEROWFORMATLINES, +)),None,None,List(),List(),None,None,false) ++- Project [a#x, b#x, c#x, null AS NULL#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'XXXX' +USING 'cat' AS (a, b, c, d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' +FROM t +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x, d#x], ScriptInputOutputSchema(List((TOK_TABLEROWFORMATFIELD,@), (TOK_TABLEROWFORMATNULL,XXXX), (TOK_TABLEROWFORMATLINES, +)),List((TOK_TABLEROWFORMATFIELD,@), (TOK_TABLEROWFORMATLINES, +)),None,None,List(),List(),None,None,false) ++- Project [a#x, b#x, c#x, null AS NULL#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS '\n' +USING 'cat' AS (a, b, c, d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' +FROM t +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x, d#x], ScriptInputOutputSchema(List((TOK_TABLEROWFORMATFIELD,@), (TOK_TABLEROWFORMATNULL, +), (TOK_TABLEROWFORMATLINES, +)),List((TOK_TABLEROWFORMATFIELD,@), (TOK_TABLEROWFORMATLINES, +)),None,None,List(),List(),None,None,false) ++- Project [a#x, b#x, c#x, null AS NULL#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' + USING 'cat' AS ( + a string, + b boolean, + c binary, + d tinyint, + e int, + f smallint, + g long, + h float, + i double, + j decimal(38, 18), + k timestamp, + l date) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' + FROM t +) tmp +-- !query analysis +Project [a#x, b#x, decode(c#x, UTF-8) AS decode(c, UTF-8)#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] ++- SubqueryAlias tmp + +- ScriptTransformation cat, [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x], ScriptInputOutputSchema(List((TOK_TABLEROWFORMATFIELD,,), (TOK_TABLEROWFORMATNULL,NULL), (TOK_TABLEROWFORMATLINES, +)),List((TOK_TABLEROWFORMATFIELD,,), (TOK_TABLEROWFORMATNULL,NULL), (TOK_TABLEROWFORMATLINES, +)),None,None,List(),List(),None,None,false) + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' + USING 'cat' AS ( + a string, + b long, + c binary, + d tinyint, + e int, + f smallint, + g long, + h float, + i double, + j decimal(38, 18), + k int, + l long) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' + FROM t +) tmp +-- !query analysis +Project [a#x, b#xL, decode(c#x, UTF-8) AS decode(c, UTF-8)#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#xL] ++- SubqueryAlias tmp + +- ScriptTransformation cat, [a#x, b#xL, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#xL], ScriptInputOutputSchema(List((TOK_TABLEROWFORMATFIELD,,), (TOK_TABLEROWFORMATNULL,NULL), (TOK_TABLEROWFORMATLINES, +)),List((TOK_TABLEROWFORMATFIELD,,), (TOK_TABLEROWFORMATNULL,NULL), (TOK_TABLEROWFORMATLINES, +)),None,None,List(),List(),None,None,false) + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) + +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] + +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + +- SubqueryAlias t + +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] + + +-- !query +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '@' + NULL DEFINED AS 'NULL' + USING 'cat' AS ( + a string, + b string, + c string, + d string, + e string, + f string, + g string, + h string, + i string, + j string, + k string, + l string) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '@' + NULL DEFINED AS 'NULL' + FROM t +) tmp +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "LINES TERMINATED BY only supports newline '\\n' right now: @" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 69, + "stopIndex" : 560, + "fragment" : "SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l)\n ROW FORMAT DELIMITED\n FIELDS TERMINATED BY ','\n LINES TERMINATED BY '@'\n NULL DEFINED AS 'NULL'\n USING 'cat' AS (\n a string,\n b string,\n c string,\n d string,\n e string,\n f string,\n g string,\n h string,\n i string,\n j string,\n k string,\n l string)\n ROW FORMAT DELIMITED\n FIELDS TERMINATED BY ','\n LINES TERMINATED BY '@'\n NULL DEFINED AS 'NULL'\n FROM t" + } ] +} + + +-- !query +SELECT TRANSFORM(b, a, CAST(c AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Project [b#x, a#x, cast(c#x as string) AS c#x] + +- Filter (a#x <= 4) + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +SELECT TRANSFORM(1, 2, 3) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Project [1 AS 1#x, 2 AS 2#x, 3 AS 3#x] + +- Filter (a#x <= 4) + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +SELECT TRANSFORM(1, 2) + USING 'cat' AS (a INT, b INT) +FROM script_trans +LIMIT 1 +-- !query analysis +GlobalLimit 1 ++- LocalLimit 1 + +- ScriptTransformation cat, [a#x, b#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) + +- Project [1 AS 1#x, 2 AS 2#x] + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +SELECT TRANSFORM( + b, a, + CASE + WHEN c > 100 THEN 1 + WHEN c < 100 THEN 2 + ELSE 3 END) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Project [b#x, a#x, CASE WHEN (c#x > 100) THEN 1 WHEN (c#x < 100) THEN 2 ELSE 3 END AS CASE WHEN (c > 100) THEN 1 WHEN (c < 100) THEN 2 ELSE 3 END#x] + +- Filter (a#x <= 4) + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +SELECT TRANSFORM(b, a, c + 1) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Project [b#x, a#x, (c#x + 1) AS (c + 1)#x] + +- Filter (a#x <= 4) + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +SELECT TRANSFORM(*) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Project [a#x, b#x, c#x] + +- Filter (a#x <= 4) + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +SELECT TRANSFORM(b, MAX(a), CAST(SUM(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +GROUP BY b +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Aggregate [b#x], [b#x, max(a#x) AS max(a)#x, cast(sum(c#x) as string) AS CAST(sum(c) AS STRING)#x] + +- Filter (a#x <= 4) + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +SELECT TRANSFORM(b, MAX(a) FILTER (WHERE a > 3), CAST(SUM(c) AS STRING)) + USING 'cat' AS (a,b,c) +FROM script_trans +WHERE a <= 4 +GROUP BY b +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Aggregate [b#x], [b#x, max(a#x) FILTER (WHERE (a#x > 3)) AS max(a) FILTER (WHERE (a > 3))#x, cast(sum(c#x) as string) AS CAST(sum(c) AS STRING)#x] + +- Filter (a#x <= 4) + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +SELECT TRANSFORM(b, MAX(a), CAST(sum(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 2 +GROUP BY b +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Aggregate [b#x], [b#x, max(a#x) AS max(a)#x, cast(sum(c#x) as string) AS CAST(sum(c) AS STRING)#x] + +- Filter (a#x <= 2) + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +SELECT TRANSFORM(b, MAX(a), CAST(SUM(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +GROUP BY b +HAVING MAX(a) > 0 +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Filter (max(a)#x > 0) + +- Aggregate [b#x], [b#x, max(a#x) AS max(a)#x, cast(sum(c#x) as string) AS CAST(sum(c) AS STRING)#x] + +- Filter (a#x <= 4) + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +SELECT TRANSFORM(b, MAX(a), CAST(SUM(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +GROUP BY b +HAVING MAX(a) > 1 +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Filter (max(a)#x > 1) + +- Aggregate [b#x], [b#x, max(a#x) AS max(a)#x, cast(sum(c#x) as string) AS CAST(sum(c) AS STRING)#x] + +- Filter (a#x <= 4) + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +SELECT TRANSFORM(b, MAX(a) OVER w, CAST(SUM(c) OVER w AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +WINDOW w AS (PARTITION BY b ORDER BY a) +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Project [b#x, max(a) OVER (PARTITION BY b ORDER BY a ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, CAST(sum(c) OVER (PARTITION BY b ORDER BY a ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS STRING)#x] + +- Project [b#x, a#x, c#x, max(a) OVER (PARTITION BY b ORDER BY a ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, _we1#xL, max(a) OVER (PARTITION BY b ORDER BY a ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, cast(_we1#xL as string) AS CAST(sum(c) OVER (PARTITION BY b ORDER BY a ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS STRING)#x] + +- Window [max(a#x) windowspecdefinition(b#x, a#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS max(a) OVER (PARTITION BY b ORDER BY a ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, sum(c#x) windowspecdefinition(b#x, a#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS _we1#xL], [b#x], [a#x ASC NULLS FIRST] + +- Project [b#x, a#x, c#x] + +- Filter (a#x <= 4) + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +SELECT TRANSFORM(b, MAX(a), CAST(SUM(c) AS STRING), myCol, myCol2) + USING 'cat' AS (a STRING, b STRING, c STRING, d ARRAY, e STRING) +FROM script_trans +LATERAL VIEW explode(array(array(1,2,3))) myTable AS myCol +LATERAL VIEW explode(myTable.myCol) myTable2 AS myCol2 +WHERE a <= 4 +GROUP BY b, myCol, myCol2 +HAVING max(a) > 1 +-- !query analysis +ScriptTransformation cat, [a#x, b#x, c#x, d#x, e#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Filter (max(a)#x > 1) + +- Aggregate [b#x, myCol#x, myCol2#x], [b#x, max(a#x) AS max(a)#x, cast(sum(c#x) as string) AS CAST(sum(c) AS STRING)#x, myCol#x, myCol2#x] + +- Filter (a#x <= 4) + +- Generate explode(myCol#x), false, mytable2, [myCol2#x] + +- Generate explode(array(array(1, 2, 3))), false, mytable, [myCol#x] + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +FROM( + FROM script_trans + SELECT TRANSFORM(a, b) + USING 'cat' AS (`a` INT, b STRING) +) t +SELECT a + 1 +-- !query analysis +Project [(a#x + 1) AS (a + 1)#x] ++- SubqueryAlias t + +- ScriptTransformation cat, [a#x, b#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) + +- Project [a#x, b#x] + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +FROM( + SELECT TRANSFORM(a, SUM(b)) + USING 'cat' AS (`a` INT, b STRING) + FROM script_trans + GROUP BY a +) t +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] + +- 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] + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +MAP k / 10 USING 'cat' AS (one) FROM (SELECT 10 AS k) +-- !query analysis +ScriptTransformation cat, [one#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Project [(cast(k#x as double) / cast(10 as double)) AS (k / 10)#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [10 AS k#x] + +- OneRowRelation + + +-- !query +FROM (SELECT 1 AS key, 100 AS value) src +MAP src.*, src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value + USING 'cat' AS (k, v, tkey, ten, one, tvalue) +-- !query analysis +ScriptTransformation cat, [k#x, v#x, tkey#x, ten#x, one#x, tvalue#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Project [key#x, value#x, key#x, cast((cast(key#x as double) / cast(10 as double)) as int) AS CAST((key / 10) AS INT)#x, cast((key#x % 10) as int) AS CAST((key % 10) AS INT)#x, value#x] + +- SubqueryAlias src + +- Project [1 AS key#x, 100 AS value#x] + +- OneRowRelation + + +-- !query +SELECT TRANSFORM(1) + USING 'cat' AS (a) +FROM script_trans +HAVING true +-- !query analysis +ScriptTransformation cat, [a#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Filter cast(true as boolean) + +- Aggregate [1 AS 1#x] + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true +-- !query analysis +SetCommand (spark.sql.legacy.parser.havingWithoutGroupByAsWhere,Some(true)) + + +-- !query +SELECT TRANSFORM(1) + USING 'cat' AS (a) +FROM script_trans +HAVING true +-- !query analysis +ScriptTransformation cat, [a#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Filter cast(true as boolean) + +- Project [1 AS 1#x] + +- SubqueryAlias script_trans + +- View (`script_trans`, [a#x,b#x,c#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x, cast(c#x as int) AS c#x] + +- Project [a#x, b#x, c#x] + +- SubqueryAlias script_trans + +- LocalRelation [a#x, b#x, c#x] + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false +-- !query analysis +SetCommand (spark.sql.legacy.parser.havingWithoutGroupByAsWhere,Some(false)) + + +-- !query +WITH temp AS ( + SELECT TRANSFORM(a) USING 'cat' AS (b string) FROM t +) +SELECT t1.b FROM temp t1 JOIN temp t2 ON t1.b = t2.b +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias temp +: +- ScriptTransformation cat, [b#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) +: +- Project [a#x] +: +- SubqueryAlias t +: +- View (`t`, [a#x,b#x,c#x,d#x,e#x,f#x,g#xL,h#x,i#x,j#x,k#x,l#x]) +: +- Project [cast(a#x as string) AS a#x, cast(b#x as boolean) AS b#x, cast(c#x as binary) AS c#x, cast(d#x as tinyint) AS d#x, cast(e#x as int) AS e#x, cast(f#x as smallint) AS f#x, cast(g#xL as bigint) AS g#xL, cast(h#x as float) AS h#x, cast(i#x as decimal(2,1)) AS i#x, cast(j#x as decimal(10,0)) AS j#x, cast(k#x as timestamp) AS k#x, cast(l#x as date) AS l#x] +: +- Project [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] +: +- SubqueryAlias t +: +- LocalRelation [a#x, b#x, c#x, d#x, e#x, f#x, g#xL, h#x, i#x, j#x, k#x, l#x] ++- Project [b#x] + +- Join Inner, (b#x = b#x) + :- SubqueryAlias t1 + : +- SubqueryAlias temp + : +- CTERelationRef xxxx, true, [b#x] + +- SubqueryAlias t2 + +- SubqueryAlias temp + +- CTERelationRef xxxx, true, [b#x] + + +-- !query +SELECT TRANSFORM(DISTINCT b, a, c) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TRANSFORM_DISTINCT_ALL", + "sqlState" : "0A000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 92, + "fragment" : "SELECT TRANSFORM(DISTINCT b, a, c)\n USING 'cat' AS (a, b, c)\nFROM script_trans\nWHERE a <= 4" + } ] +} + + +-- !query +SELECT TRANSFORM(ALL b, a, c) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TRANSFORM_DISTINCT_ALL", + "sqlState" : "0A000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "SELECT TRANSFORM(ALL b, a, c)\n USING 'cat' AS (a, b, c)\nFROM script_trans\nWHERE a <= 4" + } ] +} + + +-- !query +SELECT TRANSFORM(b AS b_1, MAX(a), CAST(sum(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 2 +GROUP BY b +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'AS'", + "hint" : "" + } +} + + +-- !query +SELECT TRANSFORM(b b_1, MAX(a), CAST(sum(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 2 +GROUP BY b +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'b_1'", + "hint" : "" + } +} + + +-- !query +SELECT TRANSFORM(b, MAX(a) AS max_a, CAST(sum(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 2 +GROUP BY b +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'AS'", + "hint" : "" + } +} + + +-- !query +FROM ( + SELECT TRANSFORM(a, b) + USING 'cat' AS (a, b) + FROM complex_trans + CLUSTER BY a +) map_output +SELECT TRANSFORM(a, b) + USING 'cat' AS (a, b) +-- !query analysis +ScriptTransformation cat, [a#x, b#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Project [a#x, b#x] + +- SubqueryAlias map_output + +- Sort [a#x ASC NULLS FIRST], false + +- RepartitionByExpression [a#x] + +- ScriptTransformation cat, [a#x, b#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) + +- Project [a#x, b#x] + +- SubqueryAlias complex_trans + +- View (`complex_trans`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias complex_trans + +- LocalRelation [a#x, b#x] + + +-- !query +FROM ( + SELECT TRANSFORM(a, b) + USING 'cat' AS (a, b) + FROM complex_trans + ORDER BY a +) map_output +SELECT TRANSFORM(a, b) + USING 'cat' AS (a, b) +-- !query analysis +ScriptTransformation cat, [a#x, b#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) ++- Project [a#x, b#x] + +- SubqueryAlias map_output + +- Sort [a#x ASC NULLS FIRST], true + +- ScriptTransformation cat, [a#x, b#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) + +- Project [a#x, b#x] + +- SubqueryAlias complex_trans + +- View (`complex_trans`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias complex_trans + +- LocalRelation [a#x, b#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/try-string-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/try-string-functions.sql.out new file mode 100644 index 0000000000000..e7dcc9970f940 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/try-string-functions.sql.out @@ -0,0 +1,247 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select try_to_binary('', 'base64') +-- !query analysis +Project [try_to_binary(, base64) AS try_to_binary(, base64)#x] ++- OneRowRelation + + +-- !query +select try_to_binary(' ', 'base64') +-- !query analysis +Project [try_to_binary( , base64) AS try_to_binary( , base64)#x] ++- OneRowRelation + + +-- !query +select try_to_binary(' ab cd ', 'base64') +-- !query analysis +Project [try_to_binary( ab cd , base64) AS try_to_binary( ab cd , base64)#x] ++- OneRowRelation + + +-- !query +select try_to_binary(' ab c=', 'base64') +-- !query analysis +Project [try_to_binary( ab c=, base64) AS try_to_binary( ab c=, base64)#x] ++- OneRowRelation + + +-- !query +select try_to_binary(' ab cdef= = ', 'base64') +-- !query analysis +Project [try_to_binary( ab cdef= = , base64) AS try_to_binary( ab cdef= = , base64)#x] ++- OneRowRelation + + +-- !query +select try_to_binary( + concat(' b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0', + 'd2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl'), 'base64') +-- !query analysis +Project [try_to_binary(concat( b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0, d2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl), base64) AS try_to_binary(concat( b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0, d2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl), base64)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('a', 'base64') +-- !query analysis +Project [try_to_binary(a, base64) AS try_to_binary(a, base64)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('a?', 'base64') +-- !query analysis +Project [try_to_binary(a?, base64) AS try_to_binary(a?, base64)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('abcde', 'base64') +-- !query analysis +Project [try_to_binary(abcde, base64) AS try_to_binary(abcde, base64)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('abcd=', 'base64') +-- !query analysis +Project [try_to_binary(abcd=, base64) AS try_to_binary(abcd=, base64)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('a===', 'base64') +-- !query analysis +Project [try_to_binary(a===, base64) AS try_to_binary(a===, base64)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('ab==f', 'base64') +-- !query analysis +Project [try_to_binary(ab==f, base64) AS try_to_binary(ab==f, base64)#x] ++- OneRowRelation + + +-- !query +select try_to_binary( + '∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β)', 'utf-8') +-- !query analysis +Project [try_to_binary(∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β), utf-8) AS try_to_binary(∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β), utf-8)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('大千世界', 'utf8') +-- !query analysis +Project [try_to_binary(大千世界, utf8) AS try_to_binary(大千世界, utf8)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('', 'utf-8') +-- !query analysis +Project [try_to_binary(, utf-8) AS try_to_binary(, utf-8)#x] ++- OneRowRelation + + +-- !query +select try_to_binary(' ', 'utf8') +-- !query analysis +Project [try_to_binary( , utf8) AS try_to_binary( , utf8)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('737472696E67') +-- !query analysis +Project [try_to_binary(737472696E67) AS try_to_binary(737472696E67)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('737472696E67', 'hex') +-- !query analysis +Project [try_to_binary(737472696E67, hex) AS try_to_binary(737472696E67, hex)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('') +-- !query analysis +Project [try_to_binary() AS try_to_binary()#x] ++- OneRowRelation + + +-- !query +select try_to_binary('1', 'hex') +-- !query analysis +Project [try_to_binary(1, hex) AS try_to_binary(1, hex)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('FF') +-- !query analysis +Project [try_to_binary(FF) AS try_to_binary(FF)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('123') +-- !query analysis +Project [try_to_binary(123) AS try_to_binary(123)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('12345') +-- !query analysis +Project [try_to_binary(12345) AS try_to_binary(12345)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('GG') +-- !query analysis +Project [try_to_binary(GG) AS try_to_binary(GG)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('01 AF', 'hex') +-- !query analysis +Project [try_to_binary(01 AF, hex) AS try_to_binary(01 AF, hex)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('abc', concat('utf', '-8')) +-- !query analysis +Project [try_to_binary(abc, concat(utf, -8)) AS try_to_binary(abc, concat(utf, -8))#x] ++- OneRowRelation + + +-- !query +select try_to_binary(' ab cdef= = ', substr('base64whynot', 0, 6)) +-- !query analysis +Project [try_to_binary( ab cdef= = , substr(base64whynot, 0, 6)) AS try_to_binary( ab cdef= = , substr(base64whynot, 0, 6))#x] ++- OneRowRelation + + +-- !query +select try_to_binary(' ab cdef= = ', replace('HEX0', '0')) +-- !query analysis +Project [try_to_binary( ab cdef= = , replace(HEX0, 0, )) AS try_to_binary( ab cdef= = , replace(HEX0, 0, ))#x] ++- OneRowRelation + + +-- !query +select try_to_binary('abc', 'Hex') +-- !query analysis +Project [try_to_binary(abc, Hex) AS try_to_binary(abc, Hex)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('abc', null) +-- !query analysis +Project [try_to_binary(abc, null) AS try_to_binary(abc, NULL)#x] ++- OneRowRelation + + +-- !query +select try_to_binary(null, 'utf-8') +-- !query analysis +Project [try_to_binary(null, utf-8) AS try_to_binary(NULL, utf-8)#x] ++- OneRowRelation + + +-- !query +select try_to_binary(null, null) +-- !query analysis +Project [try_to_binary(null, null) AS try_to_binary(NULL, NULL)#x] ++- OneRowRelation + + +-- !query +select try_to_binary(null, cast(null as string)) +-- !query analysis +Project [try_to_binary(null, cast(null as string)) AS try_to_binary(NULL, CAST(NULL AS STRING))#x] ++- OneRowRelation + + +-- !query +select try_to_binary('abc', 1) +-- !query analysis +Project [try_to_binary(abc, 1) AS try_to_binary(abc, 1)#x] ++- OneRowRelation + + +-- !query +select try_to_binary('abc', 'invalidFormat') +-- !query analysis +Project [try_to_binary(abc, invalidFormat) AS try_to_binary(abc, invalidFormat)#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/try_aggregates.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/try_aggregates.sql.out new file mode 100644 index 0000000000000..734f6a6139623 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/try_aggregates.sql.out @@ -0,0 +1,255 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_sum(col) FROM VALUES (5), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#xL] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#xL] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (NULL), (NULL) AS tab(col) +-- !query analysis +Aggregate [try_sum(cast(col#x as double)) AS try_sum(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query analysis +Aggregate [try_sum(col#xL) AS try_sum(col)#xL] ++- SubqueryAlias tab + +- LocalRelation [col#xL] + + +-- !query +SELECT try_sum(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) +-- !query analysis +Aggregate [try_sum(col#x) AS try_sum(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (5), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_sum((cast(col#x as double) / cast(0 as double))) AS try_sum((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query analysis +Aggregate [try_sum((col#x / cast(0 as decimal(1,0)))) AS try_sum((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_sum((cast(col#x as double) / cast(0 as double))) AS try_sum((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query analysis +Aggregate [try_sum((col#xL + 1)) AS try_sum((col + 1))#xL] ++- SubqueryAlias tab + +- LocalRelation [col#xL] + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query analysis +Aggregate [try_sum((col#x / 0)) AS try_sum((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query analysis +Aggregate [try_sum((col#x / 0)) AS try_sum((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (5), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (NULL), (NULL) AS tab(col) +-- !query analysis +Aggregate [try_avg(cast(col#x as double)) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query analysis +Aggregate [try_avg(col#xL) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#xL] + + +-- !query +SELECT try_avg(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) +-- !query analysis +Aggregate [try_avg(col#x) AS try_avg(col)#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (5), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_avg((cast(col#x as double) / cast(0 as double))) AS try_avg((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query analysis +Aggregate [try_avg((col#x / cast(0 as decimal(1,0)))) AS try_avg((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query analysis +Aggregate [try_avg((cast(col#x as double) / cast(0 as double))) AS try_avg((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query analysis +Aggregate [try_avg((col#xL + 1)) AS try_avg((col + 1))#x] ++- SubqueryAlias tab + +- LocalRelation [col#xL] + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query analysis +Aggregate [try_avg((col#x / 0)) AS try_avg((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query analysis +Aggregate [try_avg((col#x / 0)) AS try_avg((col / 0))#x] ++- SubqueryAlias tab + +- LocalRelation [col#x] 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 new file mode 100644 index 0000000000000..e8506a8589d8c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/try_arithmetic.sql.out @@ -0,0 +1,435 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_add(1, 1) +-- !query analysis +Project [try_add(1, 1) AS try_add(1, 1)#x] ++- OneRowRelation + + +-- !query +SELECT try_add(2147483647, 1) +-- !query analysis +Project [try_add(2147483647, 1) AS try_add(2147483647, 1)#x] ++- OneRowRelation + + +-- !query +SELECT try_add(-2147483648, -1) +-- !query analysis +Project [try_add(-2147483648, -1) AS try_add(-2147483648, -1)#x] ++- OneRowRelation + + +-- !query +SELECT try_add(9223372036854775807L, 1) +-- !query analysis +Project [try_add(9223372036854775807, 1) AS try_add(9223372036854775807, 1)#xL] ++- OneRowRelation + + +-- !query +SELECT try_add(-9223372036854775808L, -1) +-- !query analysis +Project [try_add(-9223372036854775808, -1) AS try_add(-9223372036854775808, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT try_add(1, (2147483647 + 1)) +-- !query analysis +Project [try_add(1, (2147483647 + 1)) AS try_add(1, (2147483647 + 1))#x] ++- OneRowRelation + + +-- !query +SELECT try_add(1L, (9223372036854775807L + 1L)) +-- !query analysis +Project [try_add(1, (9223372036854775807 + 1)) AS try_add(1, (9223372036854775807 + 1))#xL] ++- OneRowRelation + + +-- !query +SELECT try_add(1, 1.0 / 0.0) +-- !query analysis +Project [try_add(1, (1.0 / 0.0)) AS try_add(1, (1.0 / 0.0))#x] ++- OneRowRelation + + +-- !query +SELECT try_add(date'2021-01-01', 1) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(1, date'2021-01-01') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(date'2021-01-01', interval 2 year) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(date'2021-01-01', interval 2 second) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(interval 2 year, date'2021-01-01') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(interval 2 second, date'2021-01-01') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(timestamp_ltz'2021-01-01 00:00:00', interval 2 year) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(timestamp_ntz'2021-01-01 00:00:00', interval 2 second) +-- !query analysis +Project [try_add(2021-01-01 00:00:00, INTERVAL '02' SECOND) AS try_add(TIMESTAMP_NTZ '2021-01-01 00:00:00', INTERVAL '02' SECOND)#x] ++- OneRowRelation + + +-- !query +SELECT try_add(interval 2 year, timestamp_ltz'2021-01-01 00:00:00') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT try_add(interval 2 second, timestamp_ntz'2021-01-01 00:00:00') +-- !query analysis +Project [try_add(INTERVAL '02' SECOND, 2021-01-01 00:00:00) AS try_add(INTERVAL '02' SECOND, TIMESTAMP_NTZ '2021-01-01 00:00:00')#x] ++- OneRowRelation + + +-- !query +SELECT try_add(interval 2 year, interval 2 year) +-- !query analysis +Project [try_add(INTERVAL '2' YEAR, INTERVAL '2' YEAR) AS try_add(INTERVAL '2' YEAR, INTERVAL '2' YEAR)#x] ++- OneRowRelation + + +-- !query +SELECT try_add(interval 2 second, interval 2 second) +-- !query analysis +Project [try_add(INTERVAL '02' SECOND, INTERVAL '02' SECOND) AS try_add(INTERVAL '02' SECOND, INTERVAL '02' SECOND)#x] ++- OneRowRelation + + +-- !query +SELECT try_add(interval 2 year, interval 2 second) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2' YEAR\"", + "inputType" : "\"INTERVAL YEAR\"", + "paramIndex" : "1", + "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 analysis +Project [try_add(INTERVAL '2147483647' MONTH, INTERVAL '2' MONTH) AS try_add(INTERVAL '2147483647' MONTH, INTERVAL '2' MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT try_add(interval 106751991 day, interval 3 day) +-- !query analysis +Project [try_add(INTERVAL '106751991' DAY, INTERVAL '3' DAY) AS try_add(INTERVAL '106751991' DAY, INTERVAL '3' DAY)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(1, 0.5) +-- !query analysis +Project [try_divide(1, 0.5) AS try_divide(1, 0.5)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(1, 0) +-- !query analysis +Project [try_divide(1, 0) AS try_divide(1, 0)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(0, 0) +-- !query analysis +Project [try_divide(0, 0) AS try_divide(0, 0)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(1, (2147483647 + 1)) +-- !query analysis +Project [try_divide(1, (2147483647 + 1)) AS try_divide(1, (2147483647 + 1))#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(1L, (9223372036854775807L + 1L)) +-- !query analysis +Project [try_divide(1, (9223372036854775807 + 1)) AS try_divide(1, (9223372036854775807 + 1))#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(1, 1.0 / 0.0) +-- !query analysis +Project [try_divide(1, (1.0 / 0.0)) AS try_divide(1, (1.0 / 0.0))#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(interval 2 year, 2) +-- !query analysis +Project [try_divide(INTERVAL '2' YEAR, 2) AS try_divide(INTERVAL '2' YEAR, 2)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(interval 2 second, 2) +-- !query analysis +Project [try_divide(INTERVAL '02' SECOND, 2) AS try_divide(INTERVAL '02' SECOND, 2)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(interval 2 year, 0) +-- !query analysis +Project [try_divide(INTERVAL '2' YEAR, 0) AS try_divide(INTERVAL '2' YEAR, 0)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(interval 2 second, 0) +-- !query analysis +Project [try_divide(INTERVAL '02' SECOND, 0) AS try_divide(INTERVAL '02' SECOND, 0)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(interval 2147483647 month, 0.5) +-- !query analysis +Project [try_divide(INTERVAL '2147483647' MONTH, 0.5) AS try_divide(INTERVAL '2147483647' MONTH, 0.5)#x] ++- OneRowRelation + + +-- !query +SELECT try_divide(interval 106751991 day, 0.5) +-- !query analysis +Project [try_divide(INTERVAL '106751991' DAY, 0.5) AS try_divide(INTERVAL '106751991' DAY, 0.5)#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(1, 1) +-- !query analysis +Project [try_subtract(1, 1) AS try_subtract(1, 1)#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(2147483647, -1) +-- !query analysis +Project [try_subtract(2147483647, -1) AS try_subtract(2147483647, -1)#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(-2147483648, 1) +-- !query analysis +Project [try_subtract(-2147483648, 1) AS try_subtract(-2147483648, 1)#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(9223372036854775807L, -1) +-- !query analysis +Project [try_subtract(9223372036854775807, -1) AS try_subtract(9223372036854775807, -1)#xL] ++- OneRowRelation + + +-- !query +SELECT try_subtract(-9223372036854775808L, 1) +-- !query analysis +Project [try_subtract(-9223372036854775808, 1) AS try_subtract(-9223372036854775808, 1)#xL] ++- OneRowRelation + + +-- !query +SELECT try_subtract(1, (2147483647 + 1)) +-- !query analysis +Project [try_subtract(1, (2147483647 + 1)) AS try_subtract(1, (2147483647 + 1))#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(1L, (9223372036854775807L + 1L)) +-- !query analysis +Project [try_subtract(1, (9223372036854775807 + 1)) AS try_subtract(1, (9223372036854775807 + 1))#xL] ++- OneRowRelation + + +-- !query +SELECT try_subtract(1, 1.0 / 0.0) +-- !query analysis +Project [try_subtract(1, (1.0 / 0.0)) AS try_subtract(1, (1.0 / 0.0))#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(interval 2 year, interval 3 year) +-- !query analysis +Project [try_subtract(INTERVAL '2' YEAR, INTERVAL '3' YEAR) AS try_subtract(INTERVAL '2' YEAR, INTERVAL '3' YEAR)#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(interval 3 second, interval 2 second) +-- !query analysis +Project [try_subtract(INTERVAL '03' SECOND, INTERVAL '02' SECOND) AS try_subtract(INTERVAL '03' SECOND, INTERVAL '02' SECOND)#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(interval 2147483647 month, interval -2 month) +-- !query analysis +Project [try_subtract(INTERVAL '2147483647' MONTH, INTERVAL '-2' MONTH) AS try_subtract(INTERVAL '2147483647' MONTH, INTERVAL '-2' MONTH)#x] ++- OneRowRelation + + +-- !query +SELECT try_subtract(interval 106751991 day, interval -3 day) +-- !query analysis +Project [try_subtract(INTERVAL '106751991' DAY, INTERVAL '-3' DAY) AS try_subtract(INTERVAL '106751991' DAY, INTERVAL '-3' DAY)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(2, 3) +-- !query analysis +Project [try_multiply(2, 3) AS try_multiply(2, 3)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(2147483647, -2) +-- !query analysis +Project [try_multiply(2147483647, -2) AS try_multiply(2147483647, -2)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(-2147483648, 2) +-- !query analysis +Project [try_multiply(-2147483648, 2) AS try_multiply(-2147483648, 2)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(9223372036854775807L, 2) +-- !query analysis +Project [try_multiply(9223372036854775807, 2) AS try_multiply(9223372036854775807, 2)#xL] ++- OneRowRelation + + +-- !query +SELECT try_multiply(-9223372036854775808L, -2) +-- !query analysis +Project [try_multiply(-9223372036854775808, -2) AS try_multiply(-9223372036854775808, -2)#xL] ++- OneRowRelation + + +-- !query +SELECT try_multiply(1, (2147483647 + 1)) +-- !query analysis +Project [try_multiply(1, (2147483647 + 1)) AS try_multiply(1, (2147483647 + 1))#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(1L, (9223372036854775807L + 1L)) +-- !query analysis +Project [try_multiply(1, (9223372036854775807 + 1)) AS try_multiply(1, (9223372036854775807 + 1))#xL] ++- OneRowRelation + + +-- !query +SELECT try_multiply(1, 1.0 / 0.0) +-- !query analysis +Project [try_multiply(1, (1.0 / 0.0)) AS try_multiply(1, (1.0 / 0.0))#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(interval 2 year, 2) +-- !query analysis +Project [try_multiply(INTERVAL '2' YEAR, 2) AS try_multiply(INTERVAL '2' YEAR, 2)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(interval 2 second, 2) +-- !query analysis +Project [try_multiply(INTERVAL '02' SECOND, 2) AS try_multiply(INTERVAL '02' SECOND, 2)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(interval 2 year, 0) +-- !query analysis +Project [try_multiply(INTERVAL '2' YEAR, 0) AS try_multiply(INTERVAL '2' YEAR, 0)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(interval 2 second, 0) +-- !query analysis +Project [try_multiply(INTERVAL '02' SECOND, 0) AS try_multiply(INTERVAL '02' SECOND, 0)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(interval 2147483647 month, 2) +-- !query analysis +Project [try_multiply(INTERVAL '2147483647' MONTH, 2) AS try_multiply(INTERVAL '2147483647' MONTH, 2)#x] ++- OneRowRelation + + +-- !query +SELECT try_multiply(interval 106751991 day, 2) +-- !query analysis +Project [try_multiply(INTERVAL '106751991' DAY, 2) AS try_multiply(INTERVAL '106751991' DAY, 2)#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/try_cast.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/try_cast.sql.out new file mode 100644 index 0000000000000..5b3d7b7e702ba --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/try_cast.sql.out @@ -0,0 +1,202 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT TRY_CAST('1.23' AS int) +-- !query analysis +Project [try_cast(1.23 as int) AS TRY_CAST(1.23 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('1.23' AS long) +-- !query analysis +Project [try_cast(1.23 as bigint) AS TRY_CAST(1.23 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('-4.56' AS int) +-- !query analysis +Project [try_cast(-4.56 as int) AS TRY_CAST(-4.56 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('-4.56' AS long) +-- !query analysis +Project [try_cast(-4.56 as bigint) AS TRY_CAST(-4.56 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('abc' AS int) +-- !query analysis +Project [try_cast(abc as int) AS TRY_CAST(abc AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('abc' AS long) +-- !query analysis +Project [try_cast(abc as bigint) AS TRY_CAST(abc AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('' AS int) +-- !query analysis +Project [try_cast( as int) AS TRY_CAST( AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('' AS long) +-- !query analysis +Project [try_cast( as bigint) AS TRY_CAST( AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST(NULL AS int) +-- !query analysis +Project [try_cast(null as int) AS TRY_CAST(NULL AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST(NULL AS long) +-- !query analysis +Project [try_cast(null as bigint) AS TRY_CAST(NULL AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('123.a' AS int) +-- !query analysis +Project [try_cast(123.a as int) AS TRY_CAST(123.a AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('123.a' AS long) +-- !query analysis +Project [try_cast(123.a as bigint) AS TRY_CAST(123.a AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('-2147483648' AS int) +-- !query analysis +Project [try_cast(-2147483648 as int) AS TRY_CAST(-2147483648 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('-2147483649' AS int) +-- !query analysis +Project [try_cast(-2147483649 as int) AS TRY_CAST(-2147483649 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('2147483647' AS int) +-- !query analysis +Project [try_cast(2147483647 as int) AS TRY_CAST(2147483647 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('2147483648' AS int) +-- !query analysis +Project [try_cast(2147483648 as int) AS TRY_CAST(2147483648 AS INT)#x] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('-9223372036854775808' AS long) +-- !query analysis +Project [try_cast(-9223372036854775808 as bigint) AS TRY_CAST(-9223372036854775808 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('-9223372036854775809' AS long) +-- !query analysis +Project [try_cast(-9223372036854775809 as bigint) AS TRY_CAST(-9223372036854775809 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('9223372036854775807' AS long) +-- !query analysis +Project [try_cast(9223372036854775807 as bigint) AS TRY_CAST(9223372036854775807 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('9223372036854775808' AS long) +-- !query analysis +Project [try_cast(9223372036854775808 as bigint) AS TRY_CAST(9223372036854775808 AS BIGINT)#xL] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('interval 3 month 1 hour' AS interval) +-- !query analysis +Project [try_cast(interval 3 month 1 hour as interval) AS TRY_CAST(interval 3 month 1 hour AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST('abc' AS interval) +-- !query analysis +Project [try_cast(abc as interval) AS TRY_CAST(abc AS INTERVAL)#x] ++- OneRowRelation + + +-- !query +select TRY_CAST('true' as boolean) +-- !query analysis +Project [try_cast(true as boolean) AS TRY_CAST(true AS BOOLEAN)#x] ++- OneRowRelation + + +-- !query +select TRY_CAST('false' as boolean) +-- !query analysis +Project [try_cast(false as boolean) AS TRY_CAST(false AS BOOLEAN)#x] ++- OneRowRelation + + +-- !query +select TRY_CAST('abc' as boolean) +-- !query analysis +Project [try_cast(abc as boolean) AS TRY_CAST(abc AS BOOLEAN)#x] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST("2021-01-01" AS date) +-- !query analysis +Project [try_cast(2021-01-01 as date) AS TRY_CAST(2021-01-01 AS DATE)#x] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST("2021-101-01" AS date) +-- !query analysis +Project [try_cast(2021-101-01 as date) AS TRY_CAST(2021-101-01 AS DATE)#x] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST("2021-01-01 00:00:00" AS timestamp) +-- !query analysis +Project [try_cast(2021-01-01 00:00:00 as timestamp) AS TRY_CAST(2021-01-01 00:00:00 AS TIMESTAMP)#x] ++- OneRowRelation + + +-- !query +SELECT TRY_CAST("2021-101-01 00:00:00" AS timestamp) +-- !query analysis +Project [try_cast(2021-101-01 00:00:00 as timestamp) AS TRY_CAST(2021-101-01 00:00:00 AS TIMESTAMP)#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/try_datetime_functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/try_datetime_functions.sql.out new file mode 100644 index 0000000000000..27751e0503a61 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/try_datetime_functions.sql.out @@ -0,0 +1,41 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select try_to_timestamp(null), try_to_timestamp('2016-12-31 00:12:00'), try_to_timestamp('2016-12-31', 'yyyy-MM-dd') +-- !query analysis +Project [try_to_timestamp(cast(null as string), None, TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(NULL)#x, try_to_timestamp(2016-12-31 00:12:00, None, TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(2016-12-31 00:12:00)#x, try_to_timestamp(2016-12-31, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(2016-12-31, yyyy-MM-dd)#x] ++- OneRowRelation + + +-- !query +select try_to_timestamp(1) +-- !query analysis +Project [try_to_timestamp(1, None, TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(1)#x] ++- OneRowRelation + + +-- !query +select try_to_timestamp('2016-12-31 abc') +-- !query analysis +Project [try_to_timestamp(2016-12-31 abc, None, TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(2016-12-31 abc)#x] ++- OneRowRelation + + +-- !query +select try_to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query analysis +Project [try_to_timestamp(2019-10-06 10:11:12., Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(2019-10-06 10:11:12., yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] ++- OneRowRelation + + +-- !query +select try_to_timestamp("02-29", "MM-dd") +-- !query analysis +Project [try_to_timestamp(02-29, Some(MM-dd), TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(02-29, MM-dd)#x] ++- OneRowRelation + + +-- !query +select try_to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') +-- !query analysis +Project [try_to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEEE), TimestampType, Some(America/Los_Angeles), false) AS try_to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEEE)#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/try_element_at.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/try_element_at.sql.out new file mode 100644 index 0000000000000..2475e315884f7 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/try_element_at.sql.out @@ -0,0 +1,55 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_element_at(array(1, 2, 3), 0) +-- !query analysis +Project [try_element_at(array(1, 2, 3), 0) AS try_element_at(array(1, 2, 3), 0)#x] ++- OneRowRelation + + +-- !query +SELECT try_element_at(array(1, 2, 3), 1) +-- !query analysis +Project [try_element_at(array(1, 2, 3), 1) AS try_element_at(array(1, 2, 3), 1)#x] ++- OneRowRelation + + +-- !query +SELECT try_element_at(array(1, 2, 3), 3) +-- !query analysis +Project [try_element_at(array(1, 2, 3), 3) AS try_element_at(array(1, 2, 3), 3)#x] ++- OneRowRelation + + +-- !query +SELECT try_element_at(array(1, 2, 3), 4) +-- !query analysis +Project [try_element_at(array(1, 2, 3), 4) AS try_element_at(array(1, 2, 3), 4)#x] ++- OneRowRelation + + +-- !query +SELECT try_element_at(array(1, 2, 3), -1) +-- !query analysis +Project [try_element_at(array(1, 2, 3), -1) AS try_element_at(array(1, 2, 3), -1)#x] ++- OneRowRelation + + +-- !query +SELECT try_element_at(array(1, 2, 3), -4) +-- !query analysis +Project [try_element_at(array(1, 2, 3), -4) AS try_element_at(array(1, 2, 3), -4)#x] ++- OneRowRelation + + +-- !query +SELECT try_element_at(map('a','b'), 'a') +-- !query analysis +Project [try_element_at(map(a, b), a) AS try_element_at(map(a, b), a)#x] ++- OneRowRelation + + +-- !query +SELECT try_element_at(map('a','b'), 'abc') +-- !query analysis +Project [try_element_at(map(a, b), abc) AS try_element_at(map(a, b), abc)#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/arrayJoin.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/arrayJoin.sql.out new file mode 100644 index 0000000000000..dac3a6ea8974f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/arrayJoin.sql.out @@ -0,0 +1,74 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT array_join(array(true, false), ', ') +-- !query analysis +Project [array_join(cast(array(true, false) as array), , , None) AS array_join(array(true, false), , )#x] ++- OneRowRelation + + +-- !query +SELECT array_join(array(2Y, 1Y), ', ') +-- !query analysis +Project [array_join(cast(array(2, 1) as array), , , None) AS array_join(array(2, 1), , )#x] ++- OneRowRelation + + +-- !query +SELECT array_join(array(2S, 1S), ', ') +-- !query analysis +Project [array_join(cast(array(2, 1) as array), , , None) AS array_join(array(2, 1), , )#x] ++- OneRowRelation + + +-- !query +SELECT array_join(array(2, 1), ', ') +-- !query analysis +Project [array_join(cast(array(2, 1) as array), , , None) AS array_join(array(2, 1), , )#x] ++- OneRowRelation + + +-- !query +SELECT array_join(array(2L, 1L), ', ') +-- !query analysis +Project [array_join(cast(array(2, 1) as array), , , None) AS array_join(array(2, 1), , )#x] ++- OneRowRelation + + +-- !query +SELECT array_join(array(9223372036854775809, 9223372036854775808), ', ') +-- !query analysis +Project [array_join(cast(array(9223372036854775809, 9223372036854775808) as array), , , None) AS array_join(array(9223372036854775809, 9223372036854775808), , )#x] ++- OneRowRelation + + +-- !query +SELECT array_join(array(2.0D, 1.0D), ', ') +-- !query analysis +Project [array_join(cast(array(2.0, 1.0) as array), , , None) AS array_join(array(2.0, 1.0), , )#x] ++- OneRowRelation + + +-- !query +SELECT array_join(array(float(2.0), float(1.0)), ', ') +-- !query analysis +Project [array_join(cast(array(cast(2.0 as float), cast(1.0 as float)) as array), , , None) AS array_join(array(2.0, 1.0), , )#x] ++- OneRowRelation + + +-- !query +SELECT array_join(array(date '2016-03-14', date '2016-03-13'), ', ') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT array_join(array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), ', ') +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT array_join(array('a', 'b'), ', ') +-- !query analysis +Project [array_join(array(a, b), , , None) AS array_join(array(a, b), , )#x] ++- 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 new file mode 100644 index 0000000000000..6df9a4c21a9f9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/binaryComparison.sql.out @@ -0,0 +1,2911 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query analysis +CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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] ++- 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) > '2' FROM t +-- !query analysis +Project [(cast(1 as tinyint) > cast(2 as tinyint)) AS (CAST(1 AS TINYINT) > 2)#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) >= '2' FROM t +-- !query analysis +Project [(cast(1 as tinyint) >= cast(2 as tinyint)) AS (CAST(1 AS TINYINT) >= 2)#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) < '2' FROM t +-- !query analysis +Project [(cast(1 as tinyint) < cast(2 as tinyint)) AS (CAST(1 AS TINYINT) < 2)#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) <= '2' FROM t +-- !query analysis +Project [(cast(1 as tinyint) <= cast(2 as tinyint)) AS (CAST(1 AS TINYINT) <= 2)#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) <> '2' FROM t +-- !query analysis +Project [NOT (cast(1 as tinyint) = cast(2 as tinyint)) AS (NOT (CAST(1 AS TINYINT) = 2))#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) = 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] ++- 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) > 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] ++- 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) >= 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] ++- 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) < 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] ++- 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) <= 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] ++- 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) <> 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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- 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 smallint) = '1' FROM t +-- !query analysis +Project [(cast(1 as smallint) = cast(1 as smallint)) AS (CAST(1 AS SMALLINT) = 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 smallint) > '2' FROM t +-- !query analysis +Project [(cast(1 as smallint) > cast(2 as smallint)) AS (CAST(1 AS SMALLINT) > 2)#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 smallint) >= '2' FROM t +-- !query analysis +Project [(cast(1 as smallint) >= cast(2 as smallint)) AS (CAST(1 AS SMALLINT) >= 2)#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 smallint) < '2' FROM t +-- !query analysis +Project [(cast(1 as smallint) < cast(2 as smallint)) AS (CAST(1 AS SMALLINT) < 2)#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 smallint) <= '2' FROM t +-- !query analysis +Project [(cast(1 as smallint) <= cast(2 as smallint)) AS (CAST(1 AS SMALLINT) <= 2)#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 smallint) <> '2' FROM t +-- !query analysis +Project [NOT (cast(1 as smallint) = cast(2 as smallint)) AS (NOT (CAST(1 AS SMALLINT) = 2))#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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- 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 int) = '1' FROM t +-- !query analysis +Project [(cast(1 as int) = cast(1 as int)) AS (CAST(1 AS INT) = 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 int) > '2' FROM t +-- !query analysis +Project [(cast(1 as int) > cast(2 as int)) AS (CAST(1 AS INT) > 2)#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 int) >= '2' FROM t +-- !query analysis +Project [(cast(1 as int) >= cast(2 as int)) AS (CAST(1 AS INT) >= 2)#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 int) < '2' FROM t +-- !query analysis +Project [(cast(1 as int) < cast(2 as int)) AS (CAST(1 AS INT) < 2)#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 int) <= '2' FROM t +-- !query analysis +Project [(cast(1 as int) <= cast(2 as int)) AS (CAST(1 AS INT) <= 2)#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 int) <> '2' FROM t +-- !query analysis +Project [NOT (cast(1 as int) = cast(2 as int)) AS (NOT (CAST(1 AS INT) = 2))#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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- 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 bigint) = '1' FROM t +-- !query analysis +Project [(cast(1 as bigint) = cast(1 as bigint)) AS (CAST(1 AS BIGINT) = 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 bigint) > '2' FROM t +-- !query analysis +Project [(cast(1 as bigint) > cast(2 as bigint)) AS (CAST(1 AS BIGINT) > 2)#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 bigint) >= '2' FROM t +-- !query analysis +Project [(cast(1 as bigint) >= cast(2 as bigint)) AS (CAST(1 AS BIGINT) >= 2)#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 bigint) < '2' FROM t +-- !query analysis +Project [(cast(1 as bigint) < cast(2 as bigint)) AS (CAST(1 AS BIGINT) < 2)#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 bigint) <= '2' FROM t +-- !query analysis +Project [(cast(1 as bigint) <= cast(2 as bigint)) AS (CAST(1 AS BIGINT) <= 2)#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 bigint) <> '2' FROM t +-- !query analysis +Project [NOT (cast(1 as bigint) = cast(2 as bigint)) AS (NOT (CAST(1 AS BIGINT) = 2))#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 bigint) = cast(null as string) FROM t +-- !query analysis +Project [(cast(1 as bigint) = cast(cast(null as string) as bigint)) AS (CAST(1 AS BIGINT) = 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 + + +-- !query +SELECT cast(1 as bigint) > cast(null as string) FROM t +-- !query analysis +Project [(cast(1 as bigint) > cast(cast(null as string) as bigint)) AS (CAST(1 AS BIGINT) > 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 + + +-- !query +SELECT cast(1 as bigint) >= cast(null as string) FROM t +-- !query analysis +Project [(cast(1 as bigint) >= cast(cast(null as string) as bigint)) AS (CAST(1 AS BIGINT) >= 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 + + +-- !query +SELECT cast(1 as bigint) < cast(null as string) FROM t +-- !query analysis +Project [(cast(1 as bigint) < cast(cast(null as string) as bigint)) AS (CAST(1 AS BIGINT) < 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 + + +-- !query +SELECT cast(1 as bigint) <= cast(null as string) FROM t +-- !query analysis +Project [(cast(1 as bigint) <= cast(cast(null as string) as bigint)) AS (CAST(1 AS BIGINT) <= 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 + + +-- !query +SELECT cast(1 as bigint) <> cast(null as string) FROM t +-- !query analysis +Project [NOT (cast(1 as bigint) = cast(cast(null as string) as bigint)) AS (NOT (CAST(1 AS BIGINT) = 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 + + +-- !query +SELECT '1' = cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as bigint) = cast(1 as bigint)) AS (1 = 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 '2' > cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(2 as bigint) > cast(1 as bigint)) AS (2 > 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 '2' >= cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(2 as bigint) >= cast(1 as bigint)) AS (2 >= 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 '2' < cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(2 as bigint) < cast(1 as bigint)) AS (2 < 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 '2' <= cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(2 as bigint) <= cast(1 as bigint)) AS (2 <= 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 '2' <> cast(1 as bigint) FROM t +-- !query analysis +Project [NOT (cast(2 as bigint) = cast(1 as bigint)) AS (NOT (2 = 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 cast(null as string) = cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(null as string) as bigint) = cast(1 as bigint)) AS (CAST(NULL AS STRING) = 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 cast(null as string) > cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(null as string) as bigint) > cast(1 as bigint)) AS (CAST(NULL AS STRING) > 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 cast(null as string) >= cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(null as string) as bigint) >= cast(1 as bigint)) AS (CAST(NULL AS STRING) >= 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 cast(null as string) < cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(null as string) as bigint) < cast(1 as bigint)) AS (CAST(NULL AS STRING) < 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 cast(null as string) <= cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(null as string) as bigint) <= cast(1 as bigint)) AS (CAST(NULL AS STRING) <= 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 cast(null as string) <> cast(1 as bigint) FROM t +-- !query analysis +Project [NOT (cast(cast(null as string) as bigint) = cast(1 as bigint)) AS (NOT (CAST(NULL AS STRING) = 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 cast(1 as decimal(10, 0)) = '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) > '2' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= '2' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) < '2' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> '2' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= '2' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(null as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(null as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(null as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(null as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(null as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(null as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '2' > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '2' >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '2' < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '2' <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '2' <> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(null as string) = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(null as string) > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(null as string) >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(null as string) < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(null as string) <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(null as string) <> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as double) = '1' FROM t +-- !query analysis +Project [(cast(1 as double) = cast(1 as double)) AS (CAST(1 AS DOUBLE) = 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 double) > '2' FROM t +-- !query analysis +Project [(cast(1 as double) > cast(2 as double)) AS (CAST(1 AS DOUBLE) > 2)#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 double) >= '2' FROM t +-- !query analysis +Project [(cast(1 as double) >= cast(2 as double)) AS (CAST(1 AS DOUBLE) >= 2)#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 double) < '2' FROM t +-- !query analysis +Project [(cast(1 as double) < cast(2 as double)) AS (CAST(1 AS DOUBLE) < 2)#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 double) <= '2' FROM t +-- !query analysis +Project [(cast(1 as double) <= cast(2 as double)) AS (CAST(1 AS DOUBLE) <= 2)#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 double) <> '2' FROM t +-- !query analysis +Project [NOT (cast(1 as double) = cast(2 as double)) AS (NOT (CAST(1 AS DOUBLE) = 2))#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 double) = cast(null as string) FROM t +-- !query analysis +Project [(cast(1 as double) = cast(cast(null as string) as double)) AS (CAST(1 AS DOUBLE) = 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 + + +-- !query +SELECT cast(1 as double) > cast(null as string) FROM t +-- !query analysis +Project [(cast(1 as double) > cast(cast(null as string) as double)) AS (CAST(1 AS DOUBLE) > 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 + + +-- !query +SELECT cast(1 as double) >= cast(null as string) FROM t +-- !query analysis +Project [(cast(1 as double) >= cast(cast(null as string) as double)) AS (CAST(1 AS DOUBLE) >= 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 + + +-- !query +SELECT cast(1 as double) < cast(null as string) FROM t +-- !query analysis +Project [(cast(1 as double) < cast(cast(null as string) as double)) AS (CAST(1 AS DOUBLE) < 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 + + +-- !query +SELECT cast(1 as double) <= cast(null as string) FROM t +-- !query analysis +Project [(cast(1 as double) <= cast(cast(null as string) as double)) AS (CAST(1 AS DOUBLE) <= 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 + + +-- !query +SELECT cast(1 as double) <> cast(null as string) FROM t +-- !query analysis +Project [NOT (cast(1 as double) = cast(cast(null as string) as double)) AS (NOT (CAST(1 AS DOUBLE) = 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 + + +-- !query +SELECT '1' = cast(1 as double) FROM t +-- !query analysis +Project [(cast(1 as double) = cast(1 as double)) AS (1 = 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 '2' > cast(1 as double) FROM t +-- !query analysis +Project [(cast(2 as double) > cast(1 as double)) AS (2 > 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 '2' >= cast(1 as double) FROM t +-- !query analysis +Project [(cast(2 as double) >= cast(1 as double)) AS (2 >= 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 '2' < cast(1 as double) FROM t +-- !query analysis +Project [(cast(2 as double) < cast(1 as double)) AS (2 < 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 '2' <= cast(1 as double) FROM t +-- !query analysis +Project [(cast(2 as double) <= cast(1 as double)) AS (2 <= 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 '2' <> cast(1 as double) FROM t +-- !query analysis +Project [NOT (cast(2 as double) = cast(1 as double)) AS (NOT (2 = 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 cast(null as string) = cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(null as string) as double) = cast(1 as double)) AS (CAST(NULL AS STRING) = 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 cast(null as string) > cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(null as string) as double) > cast(1 as double)) AS (CAST(NULL AS STRING) > 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 cast(null as string) >= cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(null as string) as double) >= cast(1 as double)) AS (CAST(NULL AS STRING) >= 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 cast(null as string) < cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(null as string) as double) < cast(1 as double)) AS (CAST(NULL AS STRING) < 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 cast(null as string) <= cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(null as string) as double) <= cast(1 as double)) AS (CAST(NULL AS STRING) <= 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 cast(null as string) <> cast(1 as double) FROM t +-- !query analysis +Project [NOT (cast(cast(null as string) as double) = cast(1 as double)) AS (NOT (CAST(NULL AS STRING) = 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 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] ++- 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 float) > '2' FROM t +-- !query analysis +Project [(cast(1 as float) > cast(2 as float)) AS (CAST(1 AS FLOAT) > 2)#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 float) >= '2' FROM t +-- !query analysis +Project [(cast(1 as float) >= cast(2 as float)) AS (CAST(1 AS FLOAT) >= 2)#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 float) < '2' FROM t +-- !query analysis +Project [(cast(1 as float) < cast(2 as float)) AS (CAST(1 AS FLOAT) < 2)#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 float) <= '2' FROM t +-- !query analysis +Project [(cast(1 as float) <= cast(2 as float)) AS (CAST(1 AS FLOAT) <= 2)#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 float) <> '2' FROM t +-- !query analysis +Project [NOT (cast(1 as float) = cast(2 as float)) AS (NOT (CAST(1 AS FLOAT) = 2))#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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1996-09-09' = date('1996-09-09') FROM t +-- !query analysis +Project [(cast(1996-09-09 as date) = cast(1996-09-09 as date)) AS (1996-09-09 = 1996-09-09)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1996-9-10' > date('1996-09-09') FROM t +-- !query analysis +Project [(cast(1996-9-10 as date) > cast(1996-09-09 as date)) AS (1996-9-10 > 1996-09-09)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1996-9-10' >= date('1996-09-09') FROM t +-- !query analysis +Project [(cast(1996-9-10 as date) >= cast(1996-09-09 as date)) AS (1996-9-10 >= 1996-09-09)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1996-9-10' < date('1996-09-09') FROM t +-- !query analysis +Project [(cast(1996-9-10 as date) < cast(1996-09-09 as date)) AS (1996-9-10 < 1996-09-09)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1996-9-10' <= date('1996-09-09') FROM t +-- !query analysis +Project [(cast(1996-9-10 as date) <= cast(1996-09-09 as date)) AS (1996-9-10 <= 1996-09-09)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1996-9-10' <> date('1996-09-09') FROM t +-- !query analysis +Project [NOT (cast(1996-9-10 as date) = cast(1996-09-09 as date)) AS (NOT (1996-9-10 = 1996-09-09))#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(null as string) = date('1996-09-09') FROM t +-- !query analysis +Project [(cast(cast(null as string) as date) = cast(1996-09-09 as date)) AS (CAST(NULL AS STRING) = 1996-09-09)#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(null as string)> date('1996-09-09') FROM t +-- !query analysis +Project [(cast(cast(null as string) as date) > cast(1996-09-09 as date)) AS (CAST(NULL AS STRING) > 1996-09-09)#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(null as string)>= date('1996-09-09') FROM t +-- !query analysis +Project [(cast(cast(null as string) as date) >= cast(1996-09-09 as date)) AS (CAST(NULL AS STRING) >= 1996-09-09)#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(null as string)< date('1996-09-09') FROM t +-- !query analysis +Project [(cast(cast(null as string) as date) < cast(1996-09-09 as date)) AS (CAST(NULL AS STRING) < 1996-09-09)#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(null as string)<= date('1996-09-09') FROM t +-- !query analysis +Project [(cast(cast(null as string) as date) <= cast(1996-09-09 as date)) AS (CAST(NULL AS STRING) <= 1996-09-09)#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(null as string)<> date('1996-09-09') FROM t +-- !query analysis +Project [NOT (cast(cast(null as string) as date) = cast(1996-09-09 as date)) AS (NOT (CAST(NULL AS STRING) = 1996-09-09))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT date('1996-09-09') = '1996-09-09' FROM t +-- !query analysis +Project [(cast(1996-09-09 as date) = cast(1996-09-09 as date)) AS (1996-09-09 = 1996-09-09)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT date('1996-9-10') > '1996-09-09' FROM t +-- !query analysis +Project [(cast(1996-9-10 as date) > cast(1996-09-09 as date)) AS (1996-9-10 > 1996-09-09)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT date('1996-9-10') >= '1996-09-09' FROM t +-- !query analysis +Project [(cast(1996-9-10 as date) >= cast(1996-09-09 as date)) AS (1996-9-10 >= 1996-09-09)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT date('1996-9-10') < '1996-09-09' FROM t +-- !query analysis +Project [(cast(1996-9-10 as date) < cast(1996-09-09 as date)) AS (1996-9-10 < 1996-09-09)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT date('1996-9-10') <= '1996-09-09' FROM t +-- !query analysis +Project [(cast(1996-9-10 as date) <= cast(1996-09-09 as date)) AS (1996-9-10 <= 1996-09-09)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT date('1996-9-10') <> '1996-09-09' FROM t +-- !query analysis +Project [NOT (cast(1996-9-10 as date) = cast(1996-09-09 as date)) AS (NOT (1996-9-10 = 1996-09-09))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT date('1996-09-09') = cast(null as string) FROM t +-- !query analysis +Project [(cast(1996-09-09 as date) = cast(cast(null as string) as date)) AS (1996-09-09 = 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 + + +-- !query +SELECT date('1996-9-10') > cast(null as string) FROM t +-- !query analysis +Project [(cast(1996-9-10 as date) > cast(cast(null as string) as date)) AS (1996-9-10 > 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 + + +-- !query +SELECT date('1996-9-10') >= cast(null as string) FROM t +-- !query analysis +Project [(cast(1996-9-10 as date) >= cast(cast(null as string) as date)) AS (1996-9-10 >= 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 + + +-- !query +SELECT date('1996-9-10') < cast(null as string) FROM t +-- !query analysis +Project [(cast(1996-9-10 as date) < cast(cast(null as string) as date)) AS (1996-9-10 < 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 + + +-- !query +SELECT date('1996-9-10') <= cast(null as string) FROM t +-- !query analysis +Project [(cast(1996-9-10 as date) <= cast(cast(null as string) as date)) AS (1996-9-10 <= 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 + + +-- !query +SELECT date('1996-9-10') <> cast(null as string) FROM t +-- !query analysis +Project [NOT (cast(1996-9-10 as date) = cast(cast(null as string) as date)) AS (NOT (1996-9-10 = 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 + + +-- !query +SELECT '1996-09-09 12:12:12.4' = timestamp('1996-09-09 12:12:12.4') FROM t +-- !query analysis +Project [(cast(1996-09-09 12:12:12.4 as timestamp) = cast(1996-09-09 12:12:12.4 as timestamp)) AS (1996-09-09 12:12:12.4 = 1996-09-09 12:12:12.4)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1996-09-09 12:12:12.5' > timestamp('1996-09-09 12:12:12.4') FROM t +-- !query analysis +Project [(cast(1996-09-09 12:12:12.5 as timestamp) > cast(1996-09-09 12:12:12.4 as timestamp)) AS (1996-09-09 12:12:12.5 > 1996-09-09 12:12:12.4)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1996-09-09 12:12:12.5' >= timestamp('1996-09-09 12:12:12.4') FROM t +-- !query analysis +Project [(cast(1996-09-09 12:12:12.5 as timestamp) >= cast(1996-09-09 12:12:12.4 as timestamp)) AS (1996-09-09 12:12:12.5 >= 1996-09-09 12:12:12.4)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1996-09-09 12:12:12.5' < timestamp('1996-09-09 12:12:12.4') FROM t +-- !query analysis +Project [(cast(1996-09-09 12:12:12.5 as timestamp) < cast(1996-09-09 12:12:12.4 as timestamp)) AS (1996-09-09 12:12:12.5 < 1996-09-09 12:12:12.4)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1996-09-09 12:12:12.5' <= timestamp('1996-09-09 12:12:12.4') FROM t +-- !query analysis +Project [(cast(1996-09-09 12:12:12.5 as timestamp) <= cast(1996-09-09 12:12:12.4 as timestamp)) AS (1996-09-09 12:12:12.5 <= 1996-09-09 12:12:12.4)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1996-09-09 12:12:12.5' <> timestamp('1996-09-09 12:12:12.4') FROM t +-- !query analysis +Project [NOT (cast(1996-09-09 12:12:12.5 as timestamp) = cast(1996-09-09 12:12:12.4 as timestamp)) AS (NOT (1996-09-09 12:12:12.5 = 1996-09-09 12:12:12.4))#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(null as string) = timestamp('1996-09-09 12:12:12.4') FROM t +-- !query analysis +Project [(cast(cast(null as string) as timestamp) = cast(1996-09-09 12:12:12.4 as timestamp)) AS (CAST(NULL AS STRING) = 1996-09-09 12:12:12.4)#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(null as string) > timestamp('1996-09-09 12:12:12.4') FROM t +-- !query analysis +Project [(cast(cast(null as string) as timestamp) > cast(1996-09-09 12:12:12.4 as timestamp)) AS (CAST(NULL AS STRING) > 1996-09-09 12:12:12.4)#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(null as string) >= timestamp('1996-09-09 12:12:12.4') FROM t +-- !query analysis +Project [(cast(cast(null as string) as timestamp) >= cast(1996-09-09 12:12:12.4 as timestamp)) AS (CAST(NULL AS STRING) >= 1996-09-09 12:12:12.4)#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(null as string) < timestamp('1996-09-09 12:12:12.4') FROM t +-- !query analysis +Project [(cast(cast(null as string) as timestamp) < cast(1996-09-09 12:12:12.4 as timestamp)) AS (CAST(NULL AS STRING) < 1996-09-09 12:12:12.4)#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(null as string) <= timestamp('1996-09-09 12:12:12.4') FROM t +-- !query analysis +Project [(cast(cast(null as string) as timestamp) <= cast(1996-09-09 12:12:12.4 as timestamp)) AS (CAST(NULL AS STRING) <= 1996-09-09 12:12:12.4)#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(null as string) <> timestamp('1996-09-09 12:12:12.4') FROM t +-- !query analysis +Project [NOT (cast(cast(null as string) as timestamp) = cast(1996-09-09 12:12:12.4 as timestamp)) AS (NOT (CAST(NULL AS STRING) = 1996-09-09 12:12:12.4))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.4' )= '1996-09-09 12:12:12.4' FROM t +-- !query analysis +Project [(cast(1996-09-09 12:12:12.4 as timestamp) = cast(1996-09-09 12:12:12.4 as timestamp)) AS (1996-09-09 12:12:12.4 = 1996-09-09 12:12:12.4)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )> '1996-09-09 12:12:12.4' FROM t +-- !query analysis +Project [(cast(1996-09-09 12:12:12.5 as timestamp) > cast(1996-09-09 12:12:12.4 as timestamp)) AS (1996-09-09 12:12:12.5 > 1996-09-09 12:12:12.4)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )>= '1996-09-09 12:12:12.4' FROM t +-- !query analysis +Project [(cast(1996-09-09 12:12:12.5 as timestamp) >= cast(1996-09-09 12:12:12.4 as timestamp)) AS (1996-09-09 12:12:12.5 >= 1996-09-09 12:12:12.4)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )< '1996-09-09 12:12:12.4' FROM t +-- !query analysis +Project [(cast(1996-09-09 12:12:12.5 as timestamp) < cast(1996-09-09 12:12:12.4 as timestamp)) AS (1996-09-09 12:12:12.5 < 1996-09-09 12:12:12.4)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )<= '1996-09-09 12:12:12.4' FROM t +-- !query analysis +Project [(cast(1996-09-09 12:12:12.5 as timestamp) <= cast(1996-09-09 12:12:12.4 as timestamp)) AS (1996-09-09 12:12:12.5 <= 1996-09-09 12:12:12.4)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )<> '1996-09-09 12:12:12.4' FROM t +-- !query analysis +Project [NOT (cast(1996-09-09 12:12:12.5 as timestamp) = cast(1996-09-09 12:12:12.4 as timestamp)) AS (NOT (1996-09-09 12:12:12.5 = 1996-09-09 12:12:12.4))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.4' )= cast(null as string) FROM t +-- !query analysis +Project [(cast(1996-09-09 12:12:12.4 as timestamp) = cast(cast(null as string) as timestamp)) AS (1996-09-09 12:12:12.4 = 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 + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )> cast(null as string) FROM t +-- !query analysis +Project [(cast(1996-09-09 12:12:12.5 as timestamp) > cast(cast(null as string) as timestamp)) AS (1996-09-09 12:12:12.5 > 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 + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )>= cast(null as string) FROM t +-- !query analysis +Project [(cast(1996-09-09 12:12:12.5 as timestamp) >= cast(cast(null as string) as timestamp)) AS (1996-09-09 12:12:12.5 >= 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 + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )< cast(null as string) FROM t +-- !query analysis +Project [(cast(1996-09-09 12:12:12.5 as timestamp) < cast(cast(null as string) as timestamp)) AS (1996-09-09 12:12:12.5 < 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 + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )<= cast(null as string) FROM t +-- !query analysis +Project [(cast(1996-09-09 12:12:12.5 as timestamp) <= cast(cast(null as string) as timestamp)) AS (1996-09-09 12:12:12.5 <= 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 + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )<> cast(null as string) FROM t +-- !query analysis +Project [NOT (cast(1996-09-09 12:12:12.5 as timestamp) = cast(cast(null as string) as timestamp)) AS (NOT (1996-09-09 12:12:12.5 = 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 + + +-- !query +SELECT ' ' = X'0020' FROM t +-- !query analysis +Project [(cast( as binary) = 0x0020) AS ( = X'0020')#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT ' ' > X'001F' FROM t +-- !query analysis +Project [(cast( as binary) > 0x001F) AS ( > X'001F')#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT ' ' >= X'001F' FROM t +-- !query analysis +Project [(cast( as binary) >= 0x001F) AS ( >= X'001F')#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT ' ' < X'001F' FROM t +-- !query analysis +Project [(cast( as binary) < 0x001F) AS ( < X'001F')#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT ' ' <= X'001F' FROM t +-- !query analysis +Project [(cast( as binary) <= 0x001F) AS ( <= X'001F')#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT ' ' <> X'001F' FROM t +-- !query analysis +Project [NOT (cast( as binary) = 0x001F) AS (NOT ( = X'001F'))#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(null as string) = X'0020' FROM t +-- !query analysis +Project [(cast(cast(null as string) as binary) = 0x0020) AS (CAST(NULL AS STRING) = X'0020')#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(null as string) > X'001F' FROM t +-- !query analysis +Project [(cast(cast(null as string) as binary) > 0x001F) AS (CAST(NULL AS STRING) > X'001F')#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(null as string) >= X'001F' FROM t +-- !query analysis +Project [(cast(cast(null as string) as binary) >= 0x001F) AS (CAST(NULL AS STRING) >= X'001F')#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(null as string) < X'001F' FROM t +-- !query analysis +Project [(cast(cast(null as string) as binary) < 0x001F) AS (CAST(NULL AS STRING) < X'001F')#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(null as string) <= X'001F' FROM t +-- !query analysis +Project [(cast(cast(null as string) as binary) <= 0x001F) AS (CAST(NULL AS STRING) <= X'001F')#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(null as string) <> X'001F' FROM t +-- !query analysis +Project [NOT (cast(cast(null as string) as binary) = 0x001F) AS (NOT (CAST(NULL AS STRING) = X'001F'))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT X'0020' = ' ' FROM t +-- !query analysis +Project [(0x0020 = cast( as binary)) AS (X'0020' = )#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT X'001F' > ' ' FROM t +-- !query analysis +Project [(0x001F > cast( as binary)) AS (X'001F' > )#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT X'001F' >= ' ' FROM t +-- !query analysis +Project [(0x001F >= cast( as binary)) AS (X'001F' >= )#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT X'001F' < ' ' FROM t +-- !query analysis +Project [(0x001F < cast( as binary)) AS (X'001F' < )#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT X'001F' <= ' ' FROM t +-- !query analysis +Project [(0x001F <= cast( as binary)) AS (X'001F' <= )#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT X'001F' <> ' ' FROM t +-- !query analysis +Project [NOT (0x001F = cast( as binary)) AS (NOT (X'001F' = ))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT X'0020' = cast(null as string) FROM t +-- !query analysis +Project [(0x0020 = cast(cast(null as string) as binary)) AS (X'0020' = 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 + + +-- !query +SELECT X'001F' > cast(null as string) FROM t +-- !query analysis +Project [(0x001F > cast(cast(null as string) as binary)) AS (X'001F' > 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 + + +-- !query +SELECT X'001F' >= cast(null as string) FROM t +-- !query analysis +Project [(0x001F >= cast(cast(null as string) as binary)) AS (X'001F' >= 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 + + +-- !query +SELECT X'001F' < cast(null as string) FROM t +-- !query analysis +Project [(0x001F < cast(cast(null as string) as binary)) AS (X'001F' < 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 + + +-- !query +SELECT X'001F' <= cast(null as string) FROM t +-- !query analysis +Project [(0x001F <= cast(cast(null as string) as binary)) AS (X'001F' <= 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 + + +-- !query +SELECT X'001F' <> cast(null as string) FROM t +-- !query analysis +Project [NOT (0x001F = cast(cast(null as string) as binary)) AS (NOT (X'001F' = 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 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 new file mode 100644 index 0000000000000..2f2b27e427d0e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/booleanEquality.sql.out @@ -0,0 +1,1327 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query analysis +CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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 + + +-- !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.AnalysisException +{ + "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" : 33, + "fragment" : "true = cast('1' as binary)" + } ] +} + + +-- !query +SELECT true = cast(1 as boolean) 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 + + +-- !query +SELECT true = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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" : 56, + "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.AnalysisException +{ + "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" : 49, + "fragment" : "true = cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !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 + + +-- !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.AnalysisException +{ + "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 [(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 + + +-- !query +SELECT true <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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.AnalysisException +{ + "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 +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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !query +SELECT cast(1 as string) = true FROM t +-- !query analysis +Project [(cast(cast(1 as string) as boolean) = true) AS (CAST(1 AS STRING) = true)#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 binary) = true FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast('1' as binary) = true" + } ] +} + + +-- !query +SELECT cast(1 as boolean) = true FROM t +-- !query analysis +Project [(cast(1 as boolean) = true) AS (CAST(1 AS BOOLEAN) = true)#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.0' as timestamp) = true FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = true" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) = true FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "cast('2017-12-11 09:30:00' as date) = true" + } ] +} + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !query +SELECT cast(1 as string) <=> true FROM t +-- !query analysis +Project [(cast(cast(1 as string) as boolean) <=> true) AS (CAST(1 AS STRING) <=> true)#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 binary) <=> true FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "cast('1' as binary) <=> true" + } ] +} + + +-- !query +SELECT cast(1 as boolean) <=> true FROM t +-- !query analysis +Project [(cast(1 as boolean) <=> true) AS (CAST(1 AS BOOLEAN) <=> true)#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.0' as timestamp) <=> true FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> true" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <=> true FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> true" + } ] +} + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT false = cast('0' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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" : 34, + "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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT false = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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" : 57, + "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.AnalysisException +{ + "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" : 50, + "fragment" : "false = cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT false <=> cast('0' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT false <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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.AnalysisException +{ + "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 +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 + + +-- !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 + + +-- !query +SELECT cast(0 as string) = false FROM t +-- !query analysis +Project [(cast(cast(0 as string) as boolean) = false) AS (CAST(0 AS STRING) = 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 binary) = false FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS BINARY) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast('0' as binary) = false" + } ] +} + + +-- !query +SELECT cast(0 as boolean) = false FROM t +-- !query analysis +Project [(cast(0 as boolean) = false) AS (CAST(0 AS BOOLEAN) = 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('2017-12-11 09:30:00.0' as timestamp) = false FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = false" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) = false FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "cast('2017-12-11 09:30:00' as date) = 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 + + +-- !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 + + +-- !query +SELECT cast(0 as string) <=> false FROM t +-- !query analysis +Project [(cast(cast(0 as string) as boolean) <=> false) AS (CAST(0 AS STRING) <=> 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 binary) <=> false FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS BINARY) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "cast('0' as binary) <=> false" + } ] +} + + +-- !query +SELECT cast(0 as boolean) <=> false FROM t +-- !query analysis +Project [(cast(0 as boolean) <=> false) AS (CAST(0 AS BOOLEAN) <=> 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('2017-12-11 09:30:00.0' as timestamp) <=> false FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> false" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <=> false FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> false" + } ] +} 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 new file mode 100644 index 0000000000000..6742bb25fd9f4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/caseWhenCoercion.sql.out @@ -0,0 +1,2361 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query analysis +CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as tinyint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as tinyint) END AS CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS TINYINT) 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 tinyint) ELSE cast(2 as smallint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as tinyint) as smallint) ELSE cast(2 as smallint) END AS CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS SMALLINT) 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 tinyint) ELSE cast(2 as int) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as tinyint) as int) ELSE cast(2 as int) END AS CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS INT) 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 tinyint) ELSE cast(2 as bigint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as tinyint) as bigint) ELSE cast(2 as bigint) END AS CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS BIGINT) END#xL] ++- 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 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] ++- 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 tinyint) ELSE cast(2 as double) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as tinyint) as double) ELSE cast(2 as double) END AS CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS DOUBLE) 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 tinyint) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as tinyint) as decimal(10,0)) ELSE cast(2 as decimal(10,0)) END AS CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS DECIMAL(10,0)) 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 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] ++- 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 tinyint) ELSE cast('2' as binary) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as boolean) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 90, + "fragment" : "CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as tinyint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as smallint) ELSE cast(cast(2 as tinyint) as smallint) END AS CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS TINYINT) 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 smallint) ELSE cast(2 as smallint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as smallint) END AS CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS SMALLINT) 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 smallint) ELSE cast(2 as int) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as smallint) as int) ELSE cast(2 as int) END AS CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS INT) 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 smallint) ELSE cast(2 as bigint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as smallint) as bigint) ELSE cast(2 as bigint) END AS CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS BIGINT) END#xL] ++- 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 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] ++- 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 smallint) ELSE cast(2 as double) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as smallint) as double) ELSE cast(2 as double) END AS CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS DOUBLE) 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 smallint) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as smallint) as decimal(10,0)) ELSE cast(2 as decimal(10,0)) END AS CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS DECIMAL(10,0)) 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 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] ++- 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 smallint) ELSE cast('2' as binary) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "CASE WHEN true THEN cast(1 as smallint) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as boolean) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 98, + "fragment" : "CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 91, + "fragment" : "CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as tinyint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as int) ELSE cast(cast(2 as tinyint) as int) END AS CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS TINYINT) 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 int) ELSE cast(2 as smallint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as int) ELSE cast(cast(2 as smallint) as int) END AS CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS SMALLINT) 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 int) ELSE cast(2 as int) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as int) ELSE cast(2 as int) END AS CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS INT) 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 int) ELSE cast(2 as bigint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as int) as bigint) ELSE cast(2 as bigint) END AS CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS BIGINT) END#xL] ++- 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 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] ++- 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 int) ELSE cast(2 as double) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as int) as double) ELSE cast(2 as double) END AS CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS DOUBLE) 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 int) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as int) as decimal(10,0)) ELSE cast(2 as decimal(10,0)) END AS CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS DECIMAL(10,0)) 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 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] ++- 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 int) ELSE cast('2' as binary) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "CASE WHEN true THEN cast(1 as int) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as boolean) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "CASE WHEN true THEN cast(1 as int) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 93, + "fragment" : "CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 86, + "fragment" : "CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as tinyint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as bigint) ELSE cast(cast(2 as tinyint) as bigint) END AS CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS TINYINT) END#xL] ++- 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 bigint) ELSE cast(2 as smallint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as bigint) ELSE cast(cast(2 as smallint) as bigint) END AS CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS SMALLINT) END#xL] ++- 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 bigint) ELSE cast(2 as int) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as bigint) ELSE cast(cast(2 as int) as bigint) END AS CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS INT) END#xL] ++- 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 bigint) ELSE cast(2 as bigint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as bigint) END AS CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS BIGINT) END#xL] ++- 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 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] ++- 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 bigint) ELSE cast(2 as double) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as bigint) as double) ELSE cast(2 as double) END AS CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS DOUBLE) 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 bigint) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as bigint) as decimal(20,0)) ELSE cast(cast(2 as decimal(10,0)) as decimal(20,0)) END AS CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS DECIMAL(10,0)) 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 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] ++- 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 bigint) ELSE cast('2' as binary) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "CASE WHEN true THEN cast(1 as bigint) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as boolean) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 96, + "fragment" : "CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 89, + "fragment" : "CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 float) ELSE cast(2 as float) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as float) ELSE cast(2 as float) END AS CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS FLOAT) 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 float) ELSE cast(2 as double) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as float) as double) ELSE cast(2 as double) END AS CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS DOUBLE) 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 float) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as float) as double) ELSE cast(cast(2 as decimal(10,0)) as double) END AS CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS DECIMAL(10,0)) 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 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] ++- 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 float) ELSE cast('2' as binary) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "CASE WHEN true THEN cast(1 as float) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as boolean) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "CASE WHEN true THEN cast(1 as float) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 95, + "fragment" : "CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 88, + "fragment" : "CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as tinyint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as double) ELSE cast(cast(2 as tinyint) as double) END AS CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS TINYINT) 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 double) ELSE cast(2 as smallint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as double) ELSE cast(cast(2 as smallint) as double) END AS CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS SMALLINT) 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 double) ELSE cast(2 as int) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as double) ELSE cast(cast(2 as int) as double) END AS CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS INT) 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 double) ELSE cast(2 as bigint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as double) ELSE cast(cast(2 as bigint) as double) END AS CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS BIGINT) 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 double) ELSE cast(2 as float) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as double) ELSE cast(cast(2 as float) as double) END AS CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS FLOAT) 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 double) ELSE cast(2 as double) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as double) ELSE cast(2 as double) END AS CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS DOUBLE) 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 double) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as double) ELSE cast(cast(2 as decimal(10,0)) as double) END AS CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS DECIMAL(10,0)) 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 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] ++- 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 double) ELSE cast('2' as binary) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "CASE WHEN true THEN cast(1 as double) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as boolean) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "CASE WHEN true THEN cast(1 as double) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 96, + "fragment" : "CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 89, + "fragment" : "CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as tinyint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as decimal(10,0)) ELSE cast(cast(2 as tinyint) as decimal(10,0)) END AS CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS TINYINT) 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 decimal(10, 0)) ELSE cast(2 as smallint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as decimal(10,0)) ELSE cast(cast(2 as smallint) as decimal(10,0)) END AS CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS SMALLINT) 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 decimal(10, 0)) ELSE cast(2 as int) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as decimal(10,0)) ELSE cast(cast(2 as int) as decimal(10,0)) END AS CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS INT) 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 decimal(10, 0)) ELSE cast(2 as bigint) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as decimal(10,0)) as decimal(20,0)) ELSE cast(cast(2 as bigint) as decimal(20,0)) END AS CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS BIGINT) 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 decimal(10, 0)) ELSE cast(2 as float) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as decimal(10,0)) as double) ELSE cast(cast(2 as float) as double) END AS CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS FLOAT) 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 decimal(10, 0)) ELSE cast(2 as double) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(1 as decimal(10,0)) as double) ELSE cast(2 as double) END AS CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS DOUBLE) 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 decimal(10, 0)) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as decimal(10,0)) ELSE cast(2 as decimal(10,0)) END AS CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS DECIMAL(10,0)) 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 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] ++- 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 decimal(10, 0)) ELSE cast('2' as binary) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 81, + "fragment" : "CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as boolean) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 104, + "fragment" : "CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 string) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as string) ELSE cast(2 as string) END AS CASE WHEN true THEN CAST(1 AS STRING) 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 +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2' as binary) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as boolean) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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" + } ] +} + + +-- !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] ++- 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' 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] ++- 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 binary) ELSE cast(2 as tinyint) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"TINYINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS TINYINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as tinyint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as smallint) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"SMALLINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS SMALLINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as smallint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as int) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"INT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS INT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as int) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as bigint) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BIGINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS BIGINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as bigint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as float) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"FLOAT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS FLOAT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as float) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as double) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DOUBLE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS DOUBLE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as double) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DECIMAL(10,0)\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS DECIMAL(10,0)) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 81, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as decimal(10, 0)) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as string) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2' as binary) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as binary) ELSE cast(2 as binary) END AS CASE WHEN true THEN CAST(1 AS BINARY) 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 binary) ELSE cast(2 as boolean) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 98, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 91, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as tinyint) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"TINYINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS TINYINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as tinyint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as smallint) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"SMALLINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS SMALLINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as smallint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as int) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"INT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS INT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as int) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as bigint) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BIGINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS BIGINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as bigint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as float) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"FLOAT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS FLOAT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as float) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as double) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DOUBLE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS DOUBLE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as double) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DECIMAL(10,0)\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS DECIMAL(10,0)) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as decimal(10, 0)) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as string) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2' as binary) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as boolean) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as boolean) END AS CASE WHEN true THEN CAST(1 AS BOOLEAN) 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 boolean) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 90, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as tinyint) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TINYINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS TINYINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as tinyint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as smallint) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"SMALLINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS SMALLINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 98, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as smallint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as int) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"INT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS INT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 93, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as int) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as bigint) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BIGINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS BIGINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 96, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as bigint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as float) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"FLOAT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS FLOAT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 95, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as float) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as double) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"DOUBLE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS DOUBLE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 96, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as double) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"DECIMAL(10,0)\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS DECIMAL(10,0)) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 104, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as decimal(10, 0)) END" + } ] +} + + +-- !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] ++- 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('2017-12-12 09:30:00.0' as timestamp) ELSE cast('2' as binary) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 98, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as boolean) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(2017-12-12 09:30:00.0 as timestamp) ELSE cast(2017-12-11 09:30:00.0 as timestamp) END AS CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) 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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(2017-12-12 09:30:00.0 as timestamp) ELSE cast(cast(2017-12-11 09:30:00 as date) as timestamp) END AS CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) 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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as tinyint) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"TINYINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS TINYINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 90, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as tinyint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as smallint) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"SMALLINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS SMALLINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 91, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as smallint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as int) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"INT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS INT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 86, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as int) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as bigint) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BIGINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS BIGINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 89, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as bigint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as float) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"FLOAT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS FLOAT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 88, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as float) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as double) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DOUBLE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS DOUBLE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 89, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as double) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DECIMAL(10,0)\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS DECIMAL(10,0)) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as decimal(10, 0)) END" + } ] +} + + +-- !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] ++- 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('2017-12-12 09:30:00' as date) ELSE cast('2' as binary) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 91, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as boolean) END FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 90, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(cast(2017-12-12 09:30:00 as date) as timestamp) ELSE cast(2017-12-11 09:30:00.0 as timestamp) END AS CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) 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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query analysis +Project [CASE WHEN true THEN cast(2017-12-12 09:30:00 as date) ELSE cast(2017-12-11 09:30:00 as date) END AS CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) 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] + +- Project [1 AS 1#x] + +- OneRowRelation 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 new file mode 100644 index 0000000000000..676737a4fea8e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/concat.sql.out @@ -0,0 +1,280 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT (col1 || col2 || col3) col +FROM ( + SELECT + id col1, + string(id + 1) col2, + encode(string(id + 2), 'utf-8') col3 + FROM range(10) +) +-- !query analysis +Project [concat(concat(cast(col1#xL as string), col2#x), cast(col3#x as string)) AS col#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [id#xL AS col1#xL, 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] + +- Range (0, 10, step=1, splits=None) + + +-- !query +SELECT ((col1 || col2) || (col3 || col4) || col5) col +FROM ( + SELECT + 'prefix_' col1, + id col2, + string(id + 1) col3, + encode(string(id + 2), 'utf-8') col4, + CAST(id AS DOUBLE) col5 + FROM range(10) +) +-- !query analysis +Project [concat(concat(concat(col1#x, cast(col2#xL as string)), concat(col3#x, cast(col4#x as string))), cast(col5#x as string)) 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, splits=None) + + +-- !query +SELECT ((col1 || col2) || (col3 || col4)) col +FROM ( + SELECT + string(id) col1, + string(id + 1) col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +) +-- !query analysis +Project [concat(concat(col1#x, col2#x), cast(concat(col3#x, col4#x) as string)) 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, splits=None) + + +-- !query +set spark.sql.function.concatBinaryAsString=true +-- !query analysis +SetCommand (spark.sql.function.concatBinaryAsString,Some(true)) + + +-- !query +SELECT (col1 || col2) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2 + FROM range(10) +) +-- !query analysis +Project [concat(cast(col1#x as string), cast(col2#x as string)) 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, splits=None) + + +-- !query +SELECT (col1 || col2 || col3 || col4) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +) +-- !query analysis +Project [concat(concat(concat(cast(col1#x as string), cast(col2#x as string)), cast(col3#x as string)), cast(col4#x as string)) 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, 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, splits=None) + + +-- !query +SELECT ((col1 || col2) || (col3 || col4)) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +) +-- !query analysis +Project [concat(concat(cast(col1#x as string), cast(col2#x as string)), concat(cast(col3#x as string), cast(col4#x as string))) 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, 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, splits=None) + + +-- !query +set spark.sql.function.concatBinaryAsString=false +-- !query analysis +SetCommand (spark.sql.function.concatBinaryAsString,Some(false)) + + +-- !query +SELECT (col1 || col2) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2 + FROM range(10) +) +-- !query analysis +Project [concat(col1#x, col2#x) 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, splits=None) + + +-- !query +SELECT (col1 || col2 || col3 || col4) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +) +-- !query analysis +Project [concat(concat(concat(col1#x, col2#x), col3#x), col4#x) 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, 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, splits=None) + + +-- !query +SELECT ((col1 || col2) || (col3 || col4)) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +) +-- !query analysis +Project [concat(concat(col1#x, col2#x), concat(col3#x, col4#x)) 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, 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, splits=None) + + +-- !query +CREATE TEMPORARY VIEW various_arrays AS SELECT * FROM VALUES ( + array(true, false), array(true), + array(2Y, 1Y), array(3Y, 4Y), + array(2S, 1S), array(3S, 4S), + array(2, 1), array(3, 4), + array(2L, 1L), array(3L, 4L), + array(9223372036854775809, 9223372036854775808), array(9223372036854775808, 9223372036854775809), + array(2.0D, 1.0D), array(3.0D, 4.0D), + array(float(2.0), float(1.0)), array(float(3.0), float(4.0)), + array(date '2016-03-14', date '2016-03-13'), array(date '2016-03-12', date '2016-03-11'), + array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), + array(timestamp '2016-11-11 20:54:00.000'), + array('a', 'b'), array('c', 'd'), + array(array('a', 'b'), array('c', 'd')), array(array('e'), array('f')), + array(struct('a', 1), struct('b', 2)), array(struct('c', 3), struct('d', 4)), + array(map('a', 1), map('b', 2)), array(map('c', 3), map('d', 4)) +) AS various_arrays( + boolean_array1, boolean_array2, + tinyint_array1, tinyint_array2, + smallint_array1, smallint_array2, + int_array1, int_array2, + bigint_array1, bigint_array2, + decimal_array1, decimal_array2, + double_array1, double_array2, + float_array1, float_array2, + date_array1, data_array2, + timestamp_array1, timestamp_array2, + string_array1, string_array2, + array_array1, array_array2, + struct_array1, struct_array2, + map_array1, map_array2 +) +-- !query analysis +CreateViewCommand `various_arrays`, SELECT * FROM VALUES ( + array(true, false), array(true), + array(2Y, 1Y), array(3Y, 4Y), + array(2S, 1S), array(3S, 4S), + array(2, 1), array(3, 4), + array(2L, 1L), array(3L, 4L), + array(9223372036854775809, 9223372036854775808), array(9223372036854775808, 9223372036854775809), + array(2.0D, 1.0D), array(3.0D, 4.0D), + array(float(2.0), float(1.0)), array(float(3.0), float(4.0)), + array(date '2016-03-14', date '2016-03-13'), array(date '2016-03-12', date '2016-03-11'), + array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), + array(timestamp '2016-11-11 20:54:00.000'), + array('a', 'b'), array('c', 'd'), + array(array('a', 'b'), array('c', 'd')), array(array('e'), array('f')), + array(struct('a', 1), struct('b', 2)), array(struct('c', 3), struct('d', 4)), + array(map('a', 1), map('b', 2)), array(map('c', 3), map('d', 4)) +) AS various_arrays( + boolean_array1, boolean_array2, + tinyint_array1, tinyint_array2, + smallint_array1, smallint_array2, + int_array1, int_array2, + bigint_array1, bigint_array2, + decimal_array1, decimal_array2, + double_array1, double_array2, + float_array1, float_array2, + date_array1, data_array2, + timestamp_array1, timestamp_array2, + string_array1, string_array2, + array_array1, array_array2, + struct_array1, struct_array2, + map_array1, map_array2 +), false, false, LocalTempView, true + +- Project [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] + +- SubqueryAlias various_arrays + +- LocalRelation [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] + + +-- !query +SELECT + (boolean_array1 || boolean_array2) boolean_array, + (tinyint_array1 || tinyint_array2) tinyint_array, + (smallint_array1 || smallint_array2) smallint_array, + (int_array1 || int_array2) int_array, + (bigint_array1 || bigint_array2) bigint_array, + (decimal_array1 || decimal_array2) decimal_array, + (double_array1 || double_array2) double_array, + (float_array1 || float_array2) float_array, + (date_array1 || data_array2) data_array, + (timestamp_array1 || timestamp_array2) timestamp_array, + (string_array1 || string_array2) string_array, + (array_array1 || array_array2) array_array, + (struct_array1 || struct_array2) struct_array, + (map_array1 || map_array2) map_array +FROM various_arrays +-- !query analysis +Project [concat(boolean_array1#x, boolean_array2#x) AS boolean_array#x, concat(tinyint_array1#x, tinyint_array2#x) AS tinyint_array#x, concat(smallint_array1#x, smallint_array2#x) AS smallint_array#x, concat(int_array1#x, int_array2#x) AS int_array#x, concat(bigint_array1#x, bigint_array2#x) AS bigint_array#x, concat(decimal_array1#x, decimal_array2#x) AS decimal_array#x, concat(double_array1#x, double_array2#x) AS double_array#x, concat(float_array1#x, float_array2#x) AS float_array#x, concat(date_array1#x, data_array2#x) AS data_array#x, concat(timestamp_array1#x, timestamp_array2#x) AS timestamp_array#x, concat(string_array1#x, string_array2#x) AS string_array#x, concat(array_array1#x, array_array2#x) AS array_array#x, concat(struct_array1#x, struct_array2#x) AS struct_array#x, concat(map_array1#x, map_array2#x) AS map_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,struct_array1#x,struct_array2#x,map_array1#x,map_array2#x]) + +- 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] + +- Project [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] + +- SubqueryAlias various_arrays + +- LocalRelation [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] + + +-- !query +SELECT + (tinyint_array1 || smallint_array2) ts_array, + (smallint_array1 || int_array2) si_array, + (int_array1 || bigint_array2) ib_array, + (bigint_array1 || decimal_array2) bd_array, + (decimal_array1 || double_array2) dd_array, + (double_array1 || float_array2) df_array, + (string_array1 || data_array2) std_array, + (timestamp_array1 || string_array2) tst_array, + (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] ++- 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,struct_array1#x,struct_array2#x,map_array1#x,map_array2#x]) + +- 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] + +- Project [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] + +- SubqueryAlias various_arrays + +- LocalRelation [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] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/dateTimeOperations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/dateTimeOperations.sql.out new file mode 100644 index 0000000000000..a4d0d0fd3702e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/dateTimeOperations.sql.out @@ -0,0 +1,739 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query analysis +CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +select cast(1 as tinyint) + interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS TINYINT)\"", + "inputType" : "\"TINYINT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS TINYINT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(1 as tinyint) + interval 2 day" + } ] +} + + +-- !query +select cast(1 as smallint) + interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS SMALLINT)\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS SMALLINT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "cast(1 as smallint) + interval 2 day" + } ] +} + + +-- !query +select cast(1 as int) + interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS INT)\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS INT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "cast(1 as int) + interval 2 day" + } ] +} + + +-- !query +select cast(1 as bigint) + interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BIGINT)\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BIGINT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "cast(1 as bigint) + interval 2 day" + } ] +} + + +-- !query +select cast(1 as float) + interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS FLOAT)\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS FLOAT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "cast(1 as float) + interval 2 day" + } ] +} + + +-- !query +select cast(1 as double) + interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DOUBLE)\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS DOUBLE) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "cast(1 as double) + interval 2 day" + } ] +} + + +-- !query +select cast(1 as decimal(10, 0)) + interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS DECIMAL(10,0)) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "cast(1 as decimal(10, 0)) + interval 2 day" + } ] +} + + +-- !query +select cast('2017-12-11' as string) + interval 2 day +-- !query analysis +Project [cast(cast(cast(2017-12-11 as string) as timestamp) + INTERVAL '2' DAY as string) AS CAST(2017-12-11 AS STRING) + INTERVAL '2' DAY#x] ++- OneRowRelation + + +-- !query +select cast('2017-12-11 09:30:00' as string) + interval 2 day +-- !query analysis +Project [cast(cast(cast(2017-12-11 09:30:00 as string) as timestamp) + INTERVAL '2' DAY as string) AS CAST(2017-12-11 09:30:00 AS STRING) + INTERVAL '2' DAY#x] ++- OneRowRelation + + +-- !query +select cast('1' as binary) + interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BINARY)\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BINARY) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "cast('1' as binary) + interval 2 day" + } ] +} + + +-- !query +select cast(1 as boolean) + interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BOOLEAN)\"", + "inputType" : "\"BOOLEAN\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BOOLEAN) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(1 as boolean) + interval 2 day" + } ] +} + + +-- !query +select cast('2017-12-11 09:30:00.0' as timestamp) + interval 2 day +-- !query analysis +Project [cast(cast(2017-12-11 09:30:00.0 as timestamp) + INTERVAL '2' DAY as timestamp) AS CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + INTERVAL '2' DAY#x] ++- OneRowRelation + + +-- !query +select cast('2017-12-11 09:30:00' as date) + interval 2 day +-- !query analysis +Project [date_add(cast(2017-12-11 09:30:00 as date), extractansiintervaldays(INTERVAL '2' DAY)) AS date_add(CAST(2017-12-11 09:30:00 AS DATE), extractansiintervaldays(INTERVAL '2' DAY))#x] ++- OneRowRelation + + +-- !query +select interval 2 day + cast(1 as tinyint) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS TINYINT)\"", + "inputType" : "\"TINYINT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS TINYINT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "interval 2 day + cast(1 as tinyint)" + } ] +} + + +-- !query +select interval 2 day + cast(1 as smallint) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS SMALLINT)\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS SMALLINT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "interval 2 day + cast(1 as smallint)" + } ] +} + + +-- !query +select interval 2 day + cast(1 as int) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS INT)\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS INT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "interval 2 day + cast(1 as int)" + } ] +} + + +-- !query +select interval 2 day + cast(1 as bigint) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BIGINT)\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BIGINT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "interval 2 day + cast(1 as bigint)" + } ] +} + + +-- !query +select interval 2 day + cast(1 as float) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS FLOAT)\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS FLOAT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "interval 2 day + cast(1 as float)" + } ] +} + + +-- !query +select interval 2 day + cast(1 as double) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DOUBLE)\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS DOUBLE) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "interval 2 day + cast(1 as double)" + } ] +} + + +-- !query +select interval 2 day + cast(1 as decimal(10, 0)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS DECIMAL(10,0)) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "interval 2 day + cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +select interval 2 day + cast('2017-12-11' as string) +-- !query analysis +Project [cast(cast(cast(2017-12-11 as string) as timestamp) + INTERVAL '2' DAY as string) AS CAST(2017-12-11 AS STRING) + INTERVAL '2' DAY#x] ++- OneRowRelation + + +-- !query +select interval 2 day + cast('2017-12-11 09:30:00' as string) +-- !query analysis +Project [cast(cast(cast(2017-12-11 09:30:00 as string) as timestamp) + INTERVAL '2' DAY as string) AS CAST(2017-12-11 09:30:00 AS STRING) + INTERVAL '2' DAY#x] ++- OneRowRelation + + +-- !query +select interval 2 day + cast('1' as binary) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BINARY)\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BINARY) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "interval 2 day + cast('1' as binary)" + } ] +} + + +-- !query +select interval 2 day + cast(1 as boolean) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BOOLEAN)\"", + "inputType" : "\"BOOLEAN\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BOOLEAN) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "interval 2 day + cast(1 as boolean)" + } ] +} + + +-- !query +select interval 2 day + cast('2017-12-11 09:30:00.0' as timestamp) +-- !query analysis +Project [cast(cast(2017-12-11 09:30:00.0 as timestamp) + INTERVAL '2' DAY as timestamp) AS CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + INTERVAL '2' DAY#x] ++- OneRowRelation + + +-- !query +select interval 2 day + cast('2017-12-11 09:30:00' as date) +-- !query analysis +Project [date_add(cast(2017-12-11 09:30:00 as date), extractansiintervaldays(INTERVAL '2' DAY)) AS date_add(CAST(2017-12-11 09:30:00 AS DATE), extractansiintervaldays(INTERVAL '2' DAY))#x] ++- OneRowRelation + + +-- !query +select cast(1 as tinyint) - interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS TINYINT)\"", + "inputType" : "\"TINYINT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS TINYINT) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(1 as tinyint) - interval 2 day" + } ] +} + + +-- !query +select cast(1 as smallint) - interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS SMALLINT)\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS SMALLINT) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "cast(1 as smallint) - interval 2 day" + } ] +} + + +-- !query +select cast(1 as int) - interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS INT)\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS INT) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "cast(1 as int) - interval 2 day" + } ] +} + + +-- !query +select cast(1 as bigint) - interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BIGINT)\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BIGINT) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "cast(1 as bigint) - interval 2 day" + } ] +} + + +-- !query +select cast(1 as float) - interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS FLOAT)\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS FLOAT) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "cast(1 as float) - interval 2 day" + } ] +} + + +-- !query +select cast(1 as double) - interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DOUBLE)\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS DOUBLE) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "cast(1 as double) - interval 2 day" + } ] +} + + +-- !query +select cast(1 as decimal(10, 0)) - interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS DECIMAL(10,0)) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "cast(1 as decimal(10, 0)) - interval 2 day" + } ] +} + + +-- !query +select cast('2017-12-11' as string) - interval 2 day +-- !query analysis +Project [cast(cast(2017-12-11 as string) - INTERVAL '2' DAY as string) AS CAST(2017-12-11 AS STRING) - INTERVAL '2' DAY#x] ++- OneRowRelation + + +-- !query +select cast('2017-12-11 09:30:00' as string) - interval 2 day +-- !query analysis +Project [cast(cast(2017-12-11 09:30:00 as string) - INTERVAL '2' DAY as string) AS CAST(2017-12-11 09:30:00 AS STRING) - INTERVAL '2' DAY#x] ++- OneRowRelation + + +-- !query +select cast('1' as binary) - interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BINARY)\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BINARY) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "cast('1' as binary) - interval 2 day" + } ] +} + + +-- !query +select cast(1 as boolean) - interval 2 day +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BOOLEAN)\"", + "inputType" : "\"BOOLEAN\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BOOLEAN) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(1 as boolean) - interval 2 day" + } ] +} + + +-- !query +select cast('2017-12-11 09:30:00.0' as timestamp) - interval 2 day +-- !query analysis +Project [cast(cast(2017-12-11 09:30:00.0 as timestamp) - INTERVAL '2' DAY as timestamp) AS CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) - INTERVAL '2' DAY#x] ++- OneRowRelation + + +-- !query +select cast('2017-12-11 09:30:00' as date) - interval 2 day +-- !query analysis +Project [date_add(cast(2017-12-11 09:30:00 as date), -extractansiintervaldays(INTERVAL '2' DAY)) AS date_add(CAST(2017-12-11 09:30:00 AS DATE), (- extractansiintervaldays(INTERVAL '2' DAY)))#x] ++- OneRowRelation 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 new file mode 100644 index 0000000000000..c69150bda486d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/decimalPrecision.sql.out @@ -0,0 +1,16511 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query analysis +CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as tinyint) + cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) + cast(1 as decimal(3,0))) AS (CAST(1 AS TINYINT) + CAST(1 AS DECIMAL(3,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 cast(1 as tinyint) + cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) + cast(1 as decimal(5,0))) AS (CAST(1 AS TINYINT) + CAST(1 AS DECIMAL(5,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 cast(1 as tinyint) + cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) + cast(1 as decimal(10,0))) AS (CAST(1 AS TINYINT) + 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 cast(1 as tinyint) + cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) + cast(1 as decimal(20,0))) AS (CAST(1 AS TINYINT) + CAST(1 AS DECIMAL(20,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 cast(1 as smallint) + cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) + cast(1 as decimal(3,0))) AS (CAST(1 AS SMALLINT) + CAST(1 AS DECIMAL(3,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 cast(1 as smallint) + cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) + cast(1 as decimal(5,0))) AS (CAST(1 AS SMALLINT) + CAST(1 AS DECIMAL(5,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 cast(1 as smallint) + cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) + cast(1 as decimal(10,0))) AS (CAST(1 AS SMALLINT) + 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 cast(1 as smallint) + cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) + cast(1 as decimal(20,0))) AS (CAST(1 AS SMALLINT) + CAST(1 AS DECIMAL(20,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 cast(1 as int) + cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) + cast(1 as decimal(3,0))) AS (CAST(1 AS INT) + CAST(1 AS DECIMAL(3,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 cast(1 as int) + cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) + cast(1 as decimal(5,0))) AS (CAST(1 AS INT) + CAST(1 AS DECIMAL(5,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 cast(1 as int) + cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) + cast(1 as decimal(10,0))) AS (CAST(1 AS INT) + 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 cast(1 as int) + cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) + cast(1 as decimal(20,0))) AS (CAST(1 AS INT) + CAST(1 AS DECIMAL(20,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 cast(1 as bigint) + cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) + cast(1 as decimal(3,0))) AS (CAST(1 AS BIGINT) + CAST(1 AS DECIMAL(3,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 cast(1 as bigint) + cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) + cast(1 as decimal(5,0))) AS (CAST(1 AS BIGINT) + CAST(1 AS DECIMAL(5,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 cast(1 as bigint) + cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) + cast(1 as decimal(10,0))) AS (CAST(1 AS BIGINT) + 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 cast(1 as bigint) + cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) + cast(1 as decimal(20,0))) AS (CAST(1 AS BIGINT) + CAST(1 AS DECIMAL(20,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 cast(1 as float) + cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) + cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS FLOAT) + CAST(1 AS DECIMAL(3,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 cast(1 as float) + cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) + cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS FLOAT) + CAST(1 AS DECIMAL(5,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 cast(1 as float) + cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) + cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS FLOAT) + 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 cast(1 as float) + cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) + cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS FLOAT) + CAST(1 AS DECIMAL(20,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 cast(1 as double) + cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) + cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS DOUBLE) + CAST(1 AS DECIMAL(3,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 cast(1 as double) + cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) + cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS DOUBLE) + CAST(1 AS DECIMAL(5,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 cast(1 as double) + cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) + cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS DOUBLE) + 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 cast(1 as double) + cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) + cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS DOUBLE) + CAST(1 AS DECIMAL(20,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 cast(1 as decimal(10, 0)) + cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) + cast(1 as decimal(3,0))) AS (CAST(1 AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(3,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 cast(1 as decimal(10, 0)) + cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) + cast(1 as decimal(5,0))) AS (CAST(1 AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(5,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 cast(1 as decimal(10, 0)) + cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) + cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) + 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 cast(1 as decimal(10, 0)) + cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) + cast(1 as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(20,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 cast('1' as binary) + cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) + cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) + cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) + cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) + cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) + cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) + cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) + cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(3,0))\"", + "inputType" : "\"DECIMAL(3,0)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(5,0))\"", + "inputType" : "\"DECIMAL(5,0)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(20,0))\"", + "inputType" : "\"DECIMAL(20,0)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) + cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) + 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 + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) + cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(5,0)) + 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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) + cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(10,0)) + 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 + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) + cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(20,0)) + 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 + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) + cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(3,0)) + 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 cast(1 as decimal(5, 0)) + cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) + cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) + 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 cast(1 as decimal(10, 0)) + cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) + cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(10,0)) + 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 cast(1 as decimal(20, 0)) + cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) + cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(20,0)) + 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 cast(1 as decimal(3, 0)) + cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) + cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) + 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 cast(1 as decimal(5, 0)) + cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) + cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) + 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 cast(1 as decimal(10, 0)) + cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) + cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) + 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 cast(1 as decimal(20, 0)) + cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) + cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(20,0)) + 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 cast(1 as decimal(3, 0)) + cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) + cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(3,0)) + 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 cast(1 as decimal(5, 0)) + cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) + cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(5,0)) + 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 cast(1 as decimal(10, 0)) + cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) + cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) + 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 cast(1 as decimal(20, 0)) + cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) + cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) + 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 cast(1 as decimal(3, 0)) + cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) + cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(3,0)) + 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 cast(1 as decimal(5, 0)) + cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) + cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(5,0)) + 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 cast(1 as decimal(10, 0)) + cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) + cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(10,0)) + 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 cast(1 as decimal(20, 0)) + cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) + cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(20,0)) + 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 cast(1 as decimal(3, 0)) + cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) + cast(1 as double)) AS (CAST(1 AS DECIMAL(3,0)) + 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 cast(1 as decimal(5, 0)) + cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) + cast(1 as double)) AS (CAST(1 AS DECIMAL(5,0)) + 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 cast(1 as decimal(10, 0)) + cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) + cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) + 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 cast(1 as decimal(20, 0)) + cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) + cast(1 as double)) AS (CAST(1 AS DECIMAL(20,0)) + 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 cast(1 as decimal(3, 0)) + cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) + cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) + 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 cast(1 as decimal(5, 0)) + cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) + cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) + 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 cast(1 as decimal(10, 0)) + cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) + cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) + 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 cast(1 as decimal(20, 0)) + cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) + cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(20,0)) + 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 cast(1 as decimal(3, 0)) + cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) + cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) + cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) + cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) + cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(3,0))\"", + "inputType" : "\"DECIMAL(3,0)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(5,0))\"", + "inputType" : "\"DECIMAL(5,0)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(20,0))\"", + "inputType" : "\"DECIMAL(20,0)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) - cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) - cast(1 as decimal(3,0))) AS (CAST(1 AS TINYINT) - CAST(1 AS DECIMAL(3,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 cast(1 as tinyint) - cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) - cast(1 as decimal(5,0))) AS (CAST(1 AS TINYINT) - CAST(1 AS DECIMAL(5,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 cast(1 as tinyint) - cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) - cast(1 as decimal(10,0))) AS (CAST(1 AS TINYINT) - 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 cast(1 as tinyint) - cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) - cast(1 as decimal(20,0))) AS (CAST(1 AS TINYINT) - CAST(1 AS DECIMAL(20,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 cast(1 as smallint) - cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) - cast(1 as decimal(3,0))) AS (CAST(1 AS SMALLINT) - CAST(1 AS DECIMAL(3,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 cast(1 as smallint) - cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) - cast(1 as decimal(5,0))) AS (CAST(1 AS SMALLINT) - CAST(1 AS DECIMAL(5,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 cast(1 as smallint) - cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) - cast(1 as decimal(10,0))) AS (CAST(1 AS SMALLINT) - 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 cast(1 as smallint) - cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) - cast(1 as decimal(20,0))) AS (CAST(1 AS SMALLINT) - CAST(1 AS DECIMAL(20,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 cast(1 as int) - cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) - cast(1 as decimal(3,0))) AS (CAST(1 AS INT) - CAST(1 AS DECIMAL(3,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 cast(1 as int) - cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) - cast(1 as decimal(5,0))) AS (CAST(1 AS INT) - CAST(1 AS DECIMAL(5,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 cast(1 as int) - cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) - cast(1 as decimal(10,0))) AS (CAST(1 AS INT) - 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 cast(1 as int) - cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) - cast(1 as decimal(20,0))) AS (CAST(1 AS INT) - CAST(1 AS DECIMAL(20,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 cast(1 as bigint) - cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) - cast(1 as decimal(3,0))) AS (CAST(1 AS BIGINT) - CAST(1 AS DECIMAL(3,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 cast(1 as bigint) - cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) - cast(1 as decimal(5,0))) AS (CAST(1 AS BIGINT) - CAST(1 AS DECIMAL(5,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 cast(1 as bigint) - cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) - cast(1 as decimal(10,0))) AS (CAST(1 AS BIGINT) - 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 cast(1 as bigint) - cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) - cast(1 as decimal(20,0))) AS (CAST(1 AS BIGINT) - CAST(1 AS DECIMAL(20,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 cast(1 as float) - cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) - cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS FLOAT) - CAST(1 AS DECIMAL(3,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 cast(1 as float) - cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) - cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS FLOAT) - CAST(1 AS DECIMAL(5,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 cast(1 as float) - cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) - cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS FLOAT) - 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 cast(1 as float) - cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) - cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS FLOAT) - CAST(1 AS DECIMAL(20,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 cast(1 as double) - cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) - cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS DOUBLE) - CAST(1 AS DECIMAL(3,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 cast(1 as double) - cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) - cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS DOUBLE) - CAST(1 AS DECIMAL(5,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 cast(1 as double) - cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) - cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS DOUBLE) - 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 cast(1 as double) - cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) - cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS DOUBLE) - CAST(1 AS DECIMAL(20,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 cast(1 as decimal(10, 0)) - cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) - cast(1 as decimal(3,0))) AS (CAST(1 AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(3,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 cast(1 as decimal(10, 0)) - cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) - cast(1 as decimal(5,0))) AS (CAST(1 AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(5,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 cast(1 as decimal(10, 0)) - cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) - cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) - 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 cast(1 as decimal(10, 0)) - cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) - cast(1 as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(20,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 cast('1' as binary) - cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) - cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) - cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) - cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) - cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) - cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) - cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) - cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(3,0))\"", + "inputType" : "\"DECIMAL(3,0)\"", + "paramIndex" : "2", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) - CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(5,0))\"", + "inputType" : "\"DECIMAL(5,0)\"", + "paramIndex" : "2", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) - CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "2", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) - CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(20,0))\"", + "inputType" : "\"DECIMAL(20,0)\"", + "paramIndex" : "2", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) - CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(3,0))\"", + "inputType" : "\"DECIMAL(3,0)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(5,0))\"", + "inputType" : "\"DECIMAL(5,0)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(20,0))\"", + "inputType" : "\"DECIMAL(20,0)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) - cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) - 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 + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) - cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(5,0)) - 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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) - cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(10,0)) - 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 + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) - cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(20,0)) - 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 + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) - cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(3,0)) - 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 cast(1 as decimal(5, 0)) - cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) - cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) - 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 cast(1 as decimal(10, 0)) - cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) - cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(10,0)) - 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 cast(1 as decimal(20, 0)) - cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) - cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(20,0)) - 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 cast(1 as decimal(3, 0)) - cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) - cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) - 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 cast(1 as decimal(5, 0)) - cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) - cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) - 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 cast(1 as decimal(10, 0)) - cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) - cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) - 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 cast(1 as decimal(20, 0)) - cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) - cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(20,0)) - 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 cast(1 as decimal(3, 0)) - cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) - cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(3,0)) - 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 cast(1 as decimal(5, 0)) - cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) - cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(5,0)) - 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 cast(1 as decimal(10, 0)) - cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) - cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) - 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 cast(1 as decimal(20, 0)) - cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) - cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) - 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 cast(1 as decimal(3, 0)) - cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) - cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(3,0)) - 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 cast(1 as decimal(5, 0)) - cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) - cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(5,0)) - 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 cast(1 as decimal(10, 0)) - cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) - cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(10,0)) - 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 cast(1 as decimal(20, 0)) - cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) - cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(20,0)) - 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 cast(1 as decimal(3, 0)) - cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) - cast(1 as double)) AS (CAST(1 AS DECIMAL(3,0)) - 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 cast(1 as decimal(5, 0)) - cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) - cast(1 as double)) AS (CAST(1 AS DECIMAL(5,0)) - 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 cast(1 as decimal(10, 0)) - cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) - cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) - 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 cast(1 as decimal(20, 0)) - cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) - cast(1 as double)) AS (CAST(1 AS DECIMAL(20,0)) - 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 cast(1 as decimal(3, 0)) - cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) - cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) - 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 cast(1 as decimal(5, 0)) - cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) - cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) - 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 cast(1 as decimal(10, 0)) - cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) - cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) - 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 cast(1 as decimal(20, 0)) - cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) - cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(20,0)) - 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 cast(1 as decimal(3, 0)) - cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) - cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) - cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) - cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) - cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(3,0))\"", + "inputType" : "\"DECIMAL(3,0)\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) - CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(5,0))\"", + "inputType" : "\"DECIMAL(5,0)\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) - CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) - CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(20,0))\"", + "inputType" : "\"DECIMAL(20,0)\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) - CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(3,0))\"", + "inputType" : "\"DECIMAL(3,0)\"", + "paramIndex" : "1", + "requiredType" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) - CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(5,0))\"", + "inputType" : "\"DECIMAL(5,0)\"", + "paramIndex" : "1", + "requiredType" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) - CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "1", + "requiredType" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) - CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(20,0))\"", + "inputType" : "\"DECIMAL(20,0)\"", + "paramIndex" : "1", + "requiredType" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) - CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) * cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) * cast(1 as decimal(3,0))) AS (CAST(1 AS TINYINT) * CAST(1 AS DECIMAL(3,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 cast(1 as tinyint) * cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) * cast(1 as decimal(5,0))) AS (CAST(1 AS TINYINT) * CAST(1 AS DECIMAL(5,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 cast(1 as tinyint) * cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) * cast(1 as decimal(10,0))) AS (CAST(1 AS TINYINT) * 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 cast(1 as tinyint) * cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) * cast(1 as decimal(20,0))) AS (CAST(1 AS TINYINT) * CAST(1 AS DECIMAL(20,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 cast(1 as smallint) * cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) * cast(1 as decimal(3,0))) AS (CAST(1 AS SMALLINT) * CAST(1 AS DECIMAL(3,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 cast(1 as smallint) * cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) * cast(1 as decimal(5,0))) AS (CAST(1 AS SMALLINT) * CAST(1 AS DECIMAL(5,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 cast(1 as smallint) * cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) * cast(1 as decimal(10,0))) AS (CAST(1 AS SMALLINT) * 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 cast(1 as smallint) * cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) * cast(1 as decimal(20,0))) AS (CAST(1 AS SMALLINT) * CAST(1 AS DECIMAL(20,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 cast(1 as int) * cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) * cast(1 as decimal(3,0))) AS (CAST(1 AS INT) * CAST(1 AS DECIMAL(3,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 cast(1 as int) * cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) * cast(1 as decimal(5,0))) AS (CAST(1 AS INT) * CAST(1 AS DECIMAL(5,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 cast(1 as int) * cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) * cast(1 as decimal(10,0))) AS (CAST(1 AS INT) * 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 cast(1 as int) * cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) * cast(1 as decimal(20,0))) AS (CAST(1 AS INT) * CAST(1 AS DECIMAL(20,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 cast(1 as bigint) * cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) * cast(1 as decimal(3,0))) AS (CAST(1 AS BIGINT) * CAST(1 AS DECIMAL(3,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 cast(1 as bigint) * cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) * cast(1 as decimal(5,0))) AS (CAST(1 AS BIGINT) * CAST(1 AS DECIMAL(5,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 cast(1 as bigint) * cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) * cast(1 as decimal(10,0))) AS (CAST(1 AS BIGINT) * 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 cast(1 as bigint) * cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) * cast(1 as decimal(20,0))) AS (CAST(1 AS BIGINT) * CAST(1 AS DECIMAL(20,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 cast(1 as float) * cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) * cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS FLOAT) * CAST(1 AS DECIMAL(3,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 cast(1 as float) * cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) * cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS FLOAT) * CAST(1 AS DECIMAL(5,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 cast(1 as float) * cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) * cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS FLOAT) * 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 cast(1 as float) * cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) * cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS FLOAT) * CAST(1 AS DECIMAL(20,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 cast(1 as double) * cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) * cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS DOUBLE) * CAST(1 AS DECIMAL(3,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 cast(1 as double) * cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) * cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS DOUBLE) * CAST(1 AS DECIMAL(5,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 cast(1 as double) * cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) * cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS DOUBLE) * 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 cast(1 as double) * cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) * cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS DOUBLE) * CAST(1 AS DECIMAL(20,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 cast(1 as decimal(10, 0)) * cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) * cast(1 as decimal(3,0))) AS (CAST(1 AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(3,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 cast(1 as decimal(10, 0)) * cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) * cast(1 as decimal(5,0))) AS (CAST(1 AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(5,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 cast(1 as decimal(10, 0)) * cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) * cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) * 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 cast(1 as decimal(10, 0)) * cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) * cast(1 as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(20,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 cast('1' as binary) * cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) * cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) * cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) * cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) * cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) * cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) * cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) * cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) * cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) * 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 + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) * cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(5,0)) * 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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) * cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(10,0)) * 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 + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) * cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(20,0)) * 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 + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) * cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(3,0)) * 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 cast(1 as decimal(5, 0)) * cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) * cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) * 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 cast(1 as decimal(10, 0)) * cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) * cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(10,0)) * 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 cast(1 as decimal(20, 0)) * cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) * cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(20,0)) * 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 cast(1 as decimal(3, 0)) * cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) * cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) * 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 cast(1 as decimal(5, 0)) * cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) * cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) * 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 cast(1 as decimal(10, 0)) * cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) * cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) * 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 cast(1 as decimal(20, 0)) * cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) * cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(20,0)) * 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 cast(1 as decimal(3, 0)) * cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) * cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(3,0)) * 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 cast(1 as decimal(5, 0)) * cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) * cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(5,0)) * 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 cast(1 as decimal(10, 0)) * cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) * cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) * 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 cast(1 as decimal(20, 0)) * cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) * cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) * 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 cast(1 as decimal(3, 0)) * cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) * cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(3,0)) * 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 cast(1 as decimal(5, 0)) * cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) * cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(5,0)) * 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 cast(1 as decimal(10, 0)) * cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) * cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(10,0)) * 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 cast(1 as decimal(20, 0)) * cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) * cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(20,0)) * 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 cast(1 as decimal(3, 0)) * cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) * cast(1 as double)) AS (CAST(1 AS DECIMAL(3,0)) * 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 cast(1 as decimal(5, 0)) * cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) * cast(1 as double)) AS (CAST(1 AS DECIMAL(5,0)) * 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 cast(1 as decimal(10, 0)) * cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) * cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) * 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 cast(1 as decimal(20, 0)) * cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) * cast(1 as double)) AS (CAST(1 AS DECIMAL(20,0)) * 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 cast(1 as decimal(3, 0)) * cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) * cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) * 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 cast(1 as decimal(5, 0)) * cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) * cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) * 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 cast(1 as decimal(10, 0)) * cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) * cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) * 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 cast(1 as decimal(20, 0)) * cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) * cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(20,0)) * 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 cast(1 as decimal(3, 0)) * cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) * cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) * cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) * cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) * cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) / cast(1 as decimal(3,0))) AS (CAST(1 AS TINYINT) / CAST(1 AS DECIMAL(3,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 cast(1 as tinyint) / cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) / cast(1 as decimal(5,0))) AS (CAST(1 AS TINYINT) / CAST(1 AS DECIMAL(5,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 cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) / cast(1 as decimal(10,0))) AS (CAST(1 AS TINYINT) / 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 cast(1 as tinyint) / cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) / cast(1 as decimal(20,0))) AS (CAST(1 AS TINYINT) / CAST(1 AS DECIMAL(20,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 cast(1 as smallint) / cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) / cast(1 as decimal(3,0))) AS (CAST(1 AS SMALLINT) / CAST(1 AS DECIMAL(3,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 cast(1 as smallint) / cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) / cast(1 as decimal(5,0))) AS (CAST(1 AS SMALLINT) / CAST(1 AS DECIMAL(5,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 cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) / cast(1 as decimal(10,0))) AS (CAST(1 AS SMALLINT) / 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 cast(1 as smallint) / cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) / cast(1 as decimal(20,0))) AS (CAST(1 AS SMALLINT) / CAST(1 AS DECIMAL(20,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 cast(1 as int) / cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) / cast(1 as decimal(3,0))) AS (CAST(1 AS INT) / CAST(1 AS DECIMAL(3,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 cast(1 as int) / cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) / cast(1 as decimal(5,0))) AS (CAST(1 AS INT) / CAST(1 AS DECIMAL(5,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 cast(1 as int) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) / cast(1 as decimal(10,0))) AS (CAST(1 AS INT) / 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 cast(1 as int) / cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) / cast(1 as decimal(20,0))) AS (CAST(1 AS INT) / CAST(1 AS DECIMAL(20,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 cast(1 as bigint) / cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) / cast(1 as decimal(3,0))) AS (CAST(1 AS BIGINT) / CAST(1 AS DECIMAL(3,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 cast(1 as bigint) / cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) / cast(1 as decimal(5,0))) AS (CAST(1 AS BIGINT) / CAST(1 AS DECIMAL(5,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 cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) / cast(1 as decimal(10,0))) AS (CAST(1 AS BIGINT) / 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 cast(1 as bigint) / cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) / cast(1 as decimal(20,0))) AS (CAST(1 AS BIGINT) / CAST(1 AS DECIMAL(20,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 cast(1 as float) / cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) / cast(cast(cast(1 as decimal(3,0)) as double) as double)) AS (CAST(1 AS FLOAT) / CAST(1 AS DECIMAL(3,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 cast(1 as float) / cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) / cast(cast(cast(1 as decimal(5,0)) as double) as double)) AS (CAST(1 AS FLOAT) / CAST(1 AS DECIMAL(5,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 cast(1 as float) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) / cast(cast(cast(1 as decimal(10,0)) as double) as double)) AS (CAST(1 AS FLOAT) / 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 cast(1 as float) / cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) / cast(cast(cast(1 as decimal(20,0)) as double) as double)) AS (CAST(1 AS FLOAT) / CAST(1 AS DECIMAL(20,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 cast(1 as double) / cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) / cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS DOUBLE) / CAST(1 AS DECIMAL(3,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 cast(1 as double) / cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) / cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS DOUBLE) / CAST(1 AS DECIMAL(5,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 cast(1 as double) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) / cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS DOUBLE) / 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 cast(1 as double) / cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) / cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS DOUBLE) / CAST(1 AS DECIMAL(20,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 cast(1 as decimal(10, 0)) / cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) / cast(1 as decimal(3,0))) AS (CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(3,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 cast(1 as decimal(10, 0)) / cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) / cast(1 as decimal(5,0))) AS (CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(5,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 cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) / cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) / 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 cast(1 as decimal(10, 0)) / cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) / cast(1 as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(20,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 cast('1' as binary) / cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) / cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) / cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) / cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) / cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) / cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) / 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 + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) / cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(5,0)) / 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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) / cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(10,0)) / 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 + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) / cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(20,0)) / 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 + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) / cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(3,0)) / 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 cast(1 as decimal(5, 0)) / cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) / cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) / 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 cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) / cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(10,0)) / 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 cast(1 as decimal(20, 0)) / cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) / cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(20,0)) / 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 cast(1 as decimal(3, 0)) / cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) / cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) / 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 cast(1 as decimal(5, 0)) / cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) / cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) / 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 cast(1 as decimal(10, 0)) / cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) / cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) / 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 cast(1 as decimal(20, 0)) / cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) / cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(20,0)) / 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 cast(1 as decimal(3, 0)) / cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) / cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(3,0)) / 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 cast(1 as decimal(5, 0)) / cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) / cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(5,0)) / 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 cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) / cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) / 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 cast(1 as decimal(20, 0)) / cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) / cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) / 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 cast(1 as decimal(3, 0)) / cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) / cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(3,0)) / 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 cast(1 as decimal(5, 0)) / cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) / cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(5,0)) / 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 cast(1 as decimal(10, 0)) / cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) / cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(10,0)) / 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 cast(1 as decimal(20, 0)) / cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) / cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(20,0)) / 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 cast(1 as decimal(3, 0)) / cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) / cast(1 as double)) AS (CAST(1 AS DECIMAL(3,0)) / 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 cast(1 as decimal(5, 0)) / cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) / cast(1 as double)) AS (CAST(1 AS DECIMAL(5,0)) / 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 cast(1 as decimal(10, 0)) / cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) / cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) / 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 cast(1 as decimal(20, 0)) / cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) / cast(1 as double)) AS (CAST(1 AS DECIMAL(20,0)) / 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 cast(1 as decimal(3, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) / cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) / 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 cast(1 as decimal(5, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) / cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) / 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 cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) / cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) / 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 cast(1 as decimal(20, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) / cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(20,0)) / 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 cast(1 as decimal(3, 0)) / cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) % cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) % cast(1 as decimal(3,0))) AS (CAST(1 AS TINYINT) % CAST(1 AS DECIMAL(3,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 cast(1 as tinyint) % cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) % cast(1 as decimal(5,0))) AS (CAST(1 AS TINYINT) % CAST(1 AS DECIMAL(5,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 cast(1 as tinyint) % cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) % cast(1 as decimal(10,0))) AS (CAST(1 AS TINYINT) % 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 cast(1 as tinyint) % cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) % cast(1 as decimal(20,0))) AS (CAST(1 AS TINYINT) % CAST(1 AS DECIMAL(20,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 cast(1 as smallint) % cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) % cast(1 as decimal(3,0))) AS (CAST(1 AS SMALLINT) % CAST(1 AS DECIMAL(3,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 cast(1 as smallint) % cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) % cast(1 as decimal(5,0))) AS (CAST(1 AS SMALLINT) % CAST(1 AS DECIMAL(5,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 cast(1 as smallint) % cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) % cast(1 as decimal(10,0))) AS (CAST(1 AS SMALLINT) % 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 cast(1 as smallint) % cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) % cast(1 as decimal(20,0))) AS (CAST(1 AS SMALLINT) % CAST(1 AS DECIMAL(20,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 cast(1 as int) % cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) % cast(1 as decimal(3,0))) AS (CAST(1 AS INT) % CAST(1 AS DECIMAL(3,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 cast(1 as int) % cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) % cast(1 as decimal(5,0))) AS (CAST(1 AS INT) % CAST(1 AS DECIMAL(5,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 cast(1 as int) % cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) % cast(1 as decimal(10,0))) AS (CAST(1 AS INT) % 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 cast(1 as int) % cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) % cast(1 as decimal(20,0))) AS (CAST(1 AS INT) % CAST(1 AS DECIMAL(20,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 cast(1 as bigint) % cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) % cast(1 as decimal(3,0))) AS (CAST(1 AS BIGINT) % CAST(1 AS DECIMAL(3,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 cast(1 as bigint) % cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) % cast(1 as decimal(5,0))) AS (CAST(1 AS BIGINT) % CAST(1 AS DECIMAL(5,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 cast(1 as bigint) % cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) % cast(1 as decimal(10,0))) AS (CAST(1 AS BIGINT) % 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 cast(1 as bigint) % cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) % cast(1 as decimal(20,0))) AS (CAST(1 AS BIGINT) % CAST(1 AS DECIMAL(20,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 cast(1 as float) % cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) % cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS FLOAT) % CAST(1 AS DECIMAL(3,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 cast(1 as float) % cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) % cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS FLOAT) % CAST(1 AS DECIMAL(5,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 cast(1 as float) % cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) % cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS FLOAT) % 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 cast(1 as float) % cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) % cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS FLOAT) % CAST(1 AS DECIMAL(20,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 cast(1 as double) % cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) % cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS DOUBLE) % CAST(1 AS DECIMAL(3,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 cast(1 as double) % cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) % cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS DOUBLE) % CAST(1 AS DECIMAL(5,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 cast(1 as double) % cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) % cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS DOUBLE) % 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 cast(1 as double) % cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) % cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS DOUBLE) % CAST(1 AS DECIMAL(20,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 cast(1 as decimal(10, 0)) % cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) % cast(1 as decimal(3,0))) AS (CAST(1 AS DECIMAL(10,0)) % CAST(1 AS DECIMAL(3,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 cast(1 as decimal(10, 0)) % cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) % cast(1 as decimal(5,0))) AS (CAST(1 AS DECIMAL(10,0)) % CAST(1 AS DECIMAL(5,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 cast(1 as decimal(10, 0)) % cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) % cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) % 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 cast(1 as decimal(10, 0)) % cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) % cast(1 as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) % CAST(1 AS DECIMAL(20,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 cast('1' as binary) % cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) % cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) % cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) % cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) % cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) % cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) % cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) % cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) % cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) % 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 + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) % cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(5,0)) % 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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) % cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(10,0)) % 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 + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) % cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(20,0)) % 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 + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) % cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(3,0)) % 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 cast(1 as decimal(5, 0)) % cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) % cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) % 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 cast(1 as decimal(10, 0)) % cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) % cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(10,0)) % 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 cast(1 as decimal(20, 0)) % cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) % cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(20,0)) % 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 cast(1 as decimal(3, 0)) % cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) % cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) % 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 cast(1 as decimal(5, 0)) % cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) % cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) % 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 cast(1 as decimal(10, 0)) % cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) % cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) % 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 cast(1 as decimal(20, 0)) % cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) % cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(20,0)) % 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 cast(1 as decimal(3, 0)) % cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) % cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(3,0)) % 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 cast(1 as decimal(5, 0)) % cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) % cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(5,0)) % 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 cast(1 as decimal(10, 0)) % cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) % cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) % 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 cast(1 as decimal(20, 0)) % cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) % cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) % 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 cast(1 as decimal(3, 0)) % cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) % cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(3,0)) % 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 cast(1 as decimal(5, 0)) % cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) % cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(5,0)) % 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 cast(1 as decimal(10, 0)) % cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) % cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(10,0)) % 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 cast(1 as decimal(20, 0)) % cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) % cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(20,0)) % 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 cast(1 as decimal(3, 0)) % cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) % cast(1 as double)) AS (CAST(1 AS DECIMAL(3,0)) % 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 cast(1 as decimal(5, 0)) % cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) % cast(1 as double)) AS (CAST(1 AS DECIMAL(5,0)) % 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 cast(1 as decimal(10, 0)) % cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) % cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) % 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 cast(1 as decimal(20, 0)) % cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) % cast(1 as double)) AS (CAST(1 AS DECIMAL(20,0)) % 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 cast(1 as decimal(3, 0)) % cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) % cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) % 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 cast(1 as decimal(5, 0)) % cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) % cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) % 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 cast(1 as decimal(10, 0)) % cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) % cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) % 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 cast(1 as decimal(20, 0)) % cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) % cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(20,0)) % 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 cast(1 as decimal(3, 0)) % cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) % cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) % cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) % cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) % cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(3, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as tinyint) as decimal(3,0)), cast(1 as decimal(3,0))) AS pmod(CAST(1 AS TINYINT), CAST(1 AS DECIMAL(3,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 pmod(cast(1 as tinyint), cast(1 as decimal(5, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as tinyint) as decimal(3,0)), cast(1 as decimal(5,0))) AS pmod(CAST(1 AS TINYINT), CAST(1 AS DECIMAL(5,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 pmod(cast(1 as tinyint), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as tinyint) as decimal(3,0)), cast(1 as decimal(10,0))) AS pmod(CAST(1 AS TINYINT), 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 pmod(cast(1 as tinyint), cast(1 as decimal(20, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as tinyint) as decimal(3,0)), cast(1 as decimal(20,0))) AS pmod(CAST(1 AS TINYINT), CAST(1 AS DECIMAL(20,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 pmod(cast(1 as smallint), cast(1 as decimal(3, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as smallint) as decimal(5,0)), cast(1 as decimal(3,0))) AS pmod(CAST(1 AS SMALLINT), CAST(1 AS DECIMAL(3,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 pmod(cast(1 as smallint), cast(1 as decimal(5, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as smallint) as decimal(5,0)), cast(1 as decimal(5,0))) AS pmod(CAST(1 AS SMALLINT), CAST(1 AS DECIMAL(5,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 pmod(cast(1 as smallint), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as smallint) as decimal(5,0)), cast(1 as decimal(10,0))) AS pmod(CAST(1 AS SMALLINT), 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 pmod(cast(1 as smallint), cast(1 as decimal(20, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as smallint) as decimal(5,0)), cast(1 as decimal(20,0))) AS pmod(CAST(1 AS SMALLINT), CAST(1 AS DECIMAL(20,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 pmod(cast(1 as int), cast(1 as decimal(3, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as int) as decimal(10,0)), cast(1 as decimal(3,0))) AS pmod(CAST(1 AS INT), CAST(1 AS DECIMAL(3,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 pmod(cast(1 as int), cast(1 as decimal(5, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as int) as decimal(10,0)), cast(1 as decimal(5,0))) AS pmod(CAST(1 AS INT), CAST(1 AS DECIMAL(5,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 pmod(cast(1 as int), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as int) as decimal(10,0)), cast(1 as decimal(10,0))) AS pmod(CAST(1 AS INT), 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 pmod(cast(1 as int), cast(1 as decimal(20, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as int) as decimal(10,0)), cast(1 as decimal(20,0))) AS pmod(CAST(1 AS INT), CAST(1 AS DECIMAL(20,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 pmod(cast(1 as bigint), cast(1 as decimal(3, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as bigint) as decimal(20,0)), cast(1 as decimal(3,0))) AS pmod(CAST(1 AS BIGINT), CAST(1 AS DECIMAL(3,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 pmod(cast(1 as bigint), cast(1 as decimal(5, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as bigint) as decimal(20,0)), cast(1 as decimal(5,0))) AS pmod(CAST(1 AS BIGINT), CAST(1 AS DECIMAL(5,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 pmod(cast(1 as bigint), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as bigint) as decimal(20,0)), cast(1 as decimal(10,0))) AS pmod(CAST(1 AS BIGINT), 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 pmod(cast(1 as bigint), cast(1 as decimal(20, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as bigint) as decimal(20,0)), cast(1 as decimal(20,0))) AS pmod(CAST(1 AS BIGINT), CAST(1 AS DECIMAL(20,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 pmod(cast(1 as float), cast(1 as decimal(3, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as float) as double), cast(cast(1 as decimal(3,0)) as double)) AS pmod(CAST(1 AS FLOAT), CAST(1 AS DECIMAL(3,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 pmod(cast(1 as float), cast(1 as decimal(5, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as float) as double), cast(cast(1 as decimal(5,0)) as double)) AS pmod(CAST(1 AS FLOAT), CAST(1 AS DECIMAL(5,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 pmod(cast(1 as float), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as float) as double), cast(cast(1 as decimal(10,0)) as double)) AS pmod(CAST(1 AS FLOAT), 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 pmod(cast(1 as float), cast(1 as decimal(20, 0))) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as float) as double), cast(cast(1 as decimal(20,0)) as double)) AS pmod(CAST(1 AS FLOAT), CAST(1 AS DECIMAL(20,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 pmod(cast(1 as double), cast(1 as decimal(3, 0))) FROM t +-- !query analysis +Project [pmod(cast(1 as double), cast(cast(1 as decimal(3,0)) as double)) AS pmod(CAST(1 AS DOUBLE), CAST(1 AS DECIMAL(3,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 pmod(cast(1 as double), cast(1 as decimal(5, 0))) FROM t +-- !query analysis +Project [pmod(cast(1 as double), cast(cast(1 as decimal(5,0)) as double)) AS pmod(CAST(1 AS DOUBLE), CAST(1 AS DECIMAL(5,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 pmod(cast(1 as double), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [pmod(cast(1 as double), cast(cast(1 as decimal(10,0)) as double)) AS pmod(CAST(1 AS DOUBLE), 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 pmod(cast(1 as double), cast(1 as decimal(20, 0))) FROM t +-- !query analysis +Project [pmod(cast(1 as double), cast(cast(1 as decimal(20,0)) as double)) AS pmod(CAST(1 AS DOUBLE), CAST(1 AS DECIMAL(20,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 pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(3, 0))) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(10,0)), cast(1 as decimal(3,0))) AS pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS DECIMAL(3,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 pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(5, 0))) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(10,0)), cast(1 as decimal(5,0))) AS pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS DECIMAL(5,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 pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(10,0)), cast(1 as decimal(10,0))) AS pmod(CAST(1 AS DECIMAL(10,0)), 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 pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(20, 0))) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(10,0)), cast(1 as decimal(20,0))) AS pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS DECIMAL(20,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 pmod(cast('1' as binary), cast(1 as decimal(3, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast('1' as binary), cast(1 as decimal(3, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('1' as binary), cast(1 as decimal(5, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast('1' as binary), cast(1 as decimal(5, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('1' as binary), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast('1' as binary), cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('1' as binary), cast(1 as decimal(20, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast('1' as binary), cast(1 as decimal(20, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(3, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 81, + "fragment" : "pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(3, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(5, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 81, + "fragment" : "pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(5, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(20, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(20, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(3, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(3, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(5, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(5, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(20, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(20, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as tinyint)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(3,0)), cast(cast(1 as tinyint) as decimal(3,0))) AS pmod(CAST(1 AS DECIMAL(3,0)), 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 + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as tinyint)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(5,0)), cast(cast(1 as tinyint) as decimal(3,0))) AS pmod(CAST(1 AS DECIMAL(5,0)), 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 + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as tinyint)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(10,0)), cast(cast(1 as tinyint) as decimal(3,0))) AS pmod(CAST(1 AS DECIMAL(10,0)), 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 + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as tinyint)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(20,0)), cast(cast(1 as tinyint) as decimal(3,0))) AS pmod(CAST(1 AS DECIMAL(20,0)), 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 + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as smallint)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(3,0)), cast(cast(1 as smallint) as decimal(5,0))) AS pmod(CAST(1 AS DECIMAL(3,0)), 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 pmod(cast(1 as decimal(5, 0)) , cast(1 as smallint)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(5,0)), cast(cast(1 as smallint) as decimal(5,0))) AS pmod(CAST(1 AS DECIMAL(5,0)), 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 pmod(cast(1 as decimal(10, 0)), cast(1 as smallint)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(10,0)), cast(cast(1 as smallint) as decimal(5,0))) AS pmod(CAST(1 AS DECIMAL(10,0)), 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 pmod(cast(1 as decimal(20, 0)), cast(1 as smallint)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(20,0)), cast(cast(1 as smallint) as decimal(5,0))) AS pmod(CAST(1 AS DECIMAL(20,0)), 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 pmod(cast(1 as decimal(3, 0)) , cast(1 as int)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(3,0)), cast(cast(1 as int) as decimal(10,0))) AS pmod(CAST(1 AS DECIMAL(3,0)), 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 pmod(cast(1 as decimal(5, 0)) , cast(1 as int)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(5,0)), cast(cast(1 as int) as decimal(10,0))) AS pmod(CAST(1 AS DECIMAL(5,0)), 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 pmod(cast(1 as decimal(10, 0)), cast(1 as int)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(10,0)), cast(cast(1 as int) as decimal(10,0))) AS pmod(CAST(1 AS DECIMAL(10,0)), 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 pmod(cast(1 as decimal(20, 0)), cast(1 as int)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(20,0)), cast(cast(1 as int) as decimal(10,0))) AS pmod(CAST(1 AS DECIMAL(20,0)), 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 pmod(cast(1 as decimal(3, 0)) , cast(1 as bigint)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(3,0)), cast(cast(1 as bigint) as decimal(20,0))) AS pmod(CAST(1 AS DECIMAL(3,0)), 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 pmod(cast(1 as decimal(5, 0)) , cast(1 as bigint)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(5,0)), cast(cast(1 as bigint) as decimal(20,0))) AS pmod(CAST(1 AS DECIMAL(5,0)), 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 pmod(cast(1 as decimal(10, 0)), cast(1 as bigint)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(10,0)), cast(cast(1 as bigint) as decimal(20,0))) AS pmod(CAST(1 AS DECIMAL(10,0)), 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 pmod(cast(1 as decimal(20, 0)), cast(1 as bigint)) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(20,0)), cast(cast(1 as bigint) as decimal(20,0))) AS pmod(CAST(1 AS DECIMAL(20,0)), 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 pmod(cast(1 as decimal(3, 0)) , cast(1 as float)) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as decimal(3,0)) as double), cast(cast(1 as float) as double)) AS pmod(CAST(1 AS DECIMAL(3,0)), 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 pmod(cast(1 as decimal(5, 0)) , cast(1 as float)) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as decimal(5,0)) as double), cast(cast(1 as float) as double)) AS pmod(CAST(1 AS DECIMAL(5,0)), 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 pmod(cast(1 as decimal(10, 0)), cast(1 as float)) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as decimal(10,0)) as double), cast(cast(1 as float) as double)) AS pmod(CAST(1 AS DECIMAL(10,0)), 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 pmod(cast(1 as decimal(20, 0)), cast(1 as float)) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as decimal(20,0)) as double), cast(cast(1 as float) as double)) AS pmod(CAST(1 AS DECIMAL(20,0)), 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 pmod(cast(1 as decimal(3, 0)) , cast(1 as double)) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as decimal(3,0)) as double), cast(1 as double)) AS pmod(CAST(1 AS DECIMAL(3,0)), 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 pmod(cast(1 as decimal(5, 0)) , cast(1 as double)) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as decimal(5,0)) as double), cast(1 as double)) AS pmod(CAST(1 AS DECIMAL(5,0)), 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 pmod(cast(1 as decimal(10, 0)), cast(1 as double)) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as decimal(10,0)) as double), cast(1 as double)) AS pmod(CAST(1 AS DECIMAL(10,0)), 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 pmod(cast(1 as decimal(20, 0)), cast(1 as double)) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as decimal(20,0)) as double), cast(1 as double)) AS pmod(CAST(1 AS DECIMAL(20,0)), 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 pmod(cast(1 as decimal(3, 0)) , cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(3,0)), cast(1 as decimal(10,0))) AS pmod(CAST(1 AS DECIMAL(3,0)), 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 pmod(cast(1 as decimal(5, 0)) , cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(5,0)), cast(1 as decimal(10,0))) AS pmod(CAST(1 AS DECIMAL(5,0)), 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 pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(10,0)), cast(1 as decimal(10,0))) AS pmod(CAST(1 AS DECIMAL(10,0)), 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 pmod(cast(1 as decimal(20, 0)), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [pmod(cast(1 as decimal(20,0)), cast(1 as decimal(10,0))) AS pmod(CAST(1 AS DECIMAL(20,0)), 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 pmod(cast(1 as decimal(3, 0)) , cast(1 as string)) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as decimal(3,0)) as double), cast(cast(1 as string) as double)) AS pmod(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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as string)) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as decimal(5,0)) as double), cast(cast(1 as string) as double)) AS pmod(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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as string)) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as decimal(10,0)) as double), cast(cast(1 as string) as double)) AS pmod(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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as string)) FROM t +-- !query analysis +Project [pmod(cast(cast(1 as decimal(20,0)) as double), cast(cast(1 as string) as double)) AS pmod(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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast(1 as decimal(3, 0)) , cast('1' as binary))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast(1 as decimal(5, 0)) , cast('1' as binary))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast(1 as decimal(10, 0)), cast('1' as binary))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast(1 as decimal(20, 0)), cast('1' as binary))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast(1 as decimal(3, 0)) , cast(1 as boolean))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast(1 as decimal(5, 0)) , cast(1 as boolean))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast(1 as decimal(10, 0)), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast(1 as decimal(20, 0)), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) = cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) = cast(1 as decimal(3,0))) AS (CAST(1 AS TINYINT) = CAST(1 AS DECIMAL(3,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 cast(1 as tinyint) = cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(5,0)) = cast(1 as decimal(5,0))) AS (CAST(1 AS TINYINT) = CAST(1 AS DECIMAL(5,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 cast(1 as tinyint) = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(10,0)) = cast(1 as decimal(10,0))) AS (CAST(1 AS TINYINT) = 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 cast(1 as tinyint) = cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(20,0)) = cast(1 as decimal(20,0))) AS (CAST(1 AS TINYINT) = CAST(1 AS DECIMAL(20,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 cast(1 as smallint) = cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) = cast(cast(1 as decimal(3,0)) as decimal(5,0))) AS (CAST(1 AS SMALLINT) = CAST(1 AS DECIMAL(3,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 cast(1 as smallint) = cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) = cast(1 as decimal(5,0))) AS (CAST(1 AS SMALLINT) = CAST(1 AS DECIMAL(5,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 cast(1 as smallint) = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(10,0)) = cast(1 as decimal(10,0))) AS (CAST(1 AS SMALLINT) = 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 cast(1 as smallint) = cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(20,0)) = cast(1 as decimal(20,0))) AS (CAST(1 AS SMALLINT) = CAST(1 AS DECIMAL(20,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 cast(1 as int) = cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) = cast(cast(1 as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS INT) = CAST(1 AS DECIMAL(3,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 cast(1 as int) = cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) = cast(cast(1 as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS INT) = CAST(1 AS DECIMAL(5,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 cast(1 as int) = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) = cast(1 as decimal(10,0))) AS (CAST(1 AS INT) = 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 cast(1 as int) = cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as int) as decimal(10,0)) as decimal(20,0)) = cast(1 as decimal(20,0))) AS (CAST(1 AS INT) = CAST(1 AS DECIMAL(20,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 cast(1 as bigint) = cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) = cast(cast(1 as decimal(3,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) = CAST(1 AS DECIMAL(3,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 cast(1 as bigint) = cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) = cast(cast(1 as decimal(5,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) = CAST(1 AS DECIMAL(5,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 cast(1 as bigint) = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) = cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) = 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 cast(1 as bigint) = cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) = cast(1 as decimal(20,0))) AS (CAST(1 AS BIGINT) = CAST(1 AS DECIMAL(20,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 cast(1 as float) = cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) = cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS FLOAT) = CAST(1 AS DECIMAL(3,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 cast(1 as float) = cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) = cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS FLOAT) = CAST(1 AS DECIMAL(5,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 cast(1 as float) = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) = cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS FLOAT) = 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 cast(1 as float) = cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) = cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS FLOAT) = CAST(1 AS DECIMAL(20,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 cast(1 as double) = cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) = cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS DOUBLE) = CAST(1 AS DECIMAL(3,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 cast(1 as double) = cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) = cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS DOUBLE) = CAST(1 AS DECIMAL(5,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 cast(1 as double) = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) = cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS DOUBLE) = 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 cast(1 as double) = cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) = cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS DOUBLE) = CAST(1 AS DECIMAL(20,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 cast(1 as decimal(10, 0)) = cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) = cast(cast(1 as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(3,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 cast(1 as decimal(10, 0)) = cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) = cast(cast(1 as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(5,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 cast(1 as decimal(10, 0)) = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) = cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) = 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 cast(1 as decimal(10, 0)) = cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) = cast(1 as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(20,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 cast('1' as binary) = cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) = cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) = cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) = cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) = cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) = cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) = cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) = cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) = 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 + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) = cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) = 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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) = cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) = 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 + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) = cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) = 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 + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(5,0)) = cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(3,0)) = 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 cast(1 as decimal(5, 0)) = cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) = cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) = 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 cast(1 as decimal(10, 0)) = cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) = cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) = 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 cast(1 as decimal(20, 0)) = cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) = cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) = 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 cast(1 as decimal(3, 0)) = cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(10,0)) = cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) = 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 cast(1 as decimal(5, 0)) = cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(10,0)) = cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) = 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 cast(1 as decimal(10, 0)) = cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) = cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) = 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 cast(1 as decimal(20, 0)) = cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) = cast(cast(cast(1 as int) as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) = 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 cast(1 as decimal(3, 0)) = cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(20,0)) = cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(3,0)) = 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 cast(1 as decimal(5, 0)) = cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(20,0)) = cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(5,0)) = 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 cast(1 as decimal(10, 0)) = cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) = cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) = 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 cast(1 as decimal(20, 0)) = cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) = cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) = 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 cast(1 as decimal(3, 0)) = cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) = cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(3,0)) = 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 cast(1 as decimal(5, 0)) = cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) = cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(5,0)) = 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 cast(1 as decimal(10, 0)) = cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) = cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(10,0)) = 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 cast(1 as decimal(20, 0)) = cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) = cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(20,0)) = 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 cast(1 as decimal(3, 0)) = cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) = cast(1 as double)) AS (CAST(1 AS DECIMAL(3,0)) = 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 cast(1 as decimal(5, 0)) = cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) = cast(1 as double)) AS (CAST(1 AS DECIMAL(5,0)) = 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 cast(1 as decimal(10, 0)) = cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) = cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) = 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 cast(1 as decimal(20, 0)) = cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) = cast(1 as double)) AS (CAST(1 AS DECIMAL(20,0)) = 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 cast(1 as decimal(3, 0)) = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(10,0)) = cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) = 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 cast(1 as decimal(5, 0)) = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(10,0)) = cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) = 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 cast(1 as decimal(10, 0)) = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) = cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) = 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 cast(1 as decimal(20, 0)) = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) = cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) = 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 cast(1 as decimal(3, 0)) = cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) = cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) = cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) = cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) = cast('1' as binary)" + } ] +} + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) <=> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) <=> cast(1 as decimal(3,0))) AS (CAST(1 AS TINYINT) <=> CAST(1 AS DECIMAL(3,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 cast(1 as tinyint) <=> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(5,0)) <=> cast(1 as decimal(5,0))) AS (CAST(1 AS TINYINT) <=> CAST(1 AS DECIMAL(5,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 cast(1 as tinyint) <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(10,0)) <=> cast(1 as decimal(10,0))) AS (CAST(1 AS TINYINT) <=> 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 cast(1 as tinyint) <=> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(20,0)) <=> cast(1 as decimal(20,0))) AS (CAST(1 AS TINYINT) <=> CAST(1 AS DECIMAL(20,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 cast(1 as smallint) <=> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) <=> cast(cast(1 as decimal(3,0)) as decimal(5,0))) AS (CAST(1 AS SMALLINT) <=> CAST(1 AS DECIMAL(3,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 cast(1 as smallint) <=> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) <=> cast(1 as decimal(5,0))) AS (CAST(1 AS SMALLINT) <=> CAST(1 AS DECIMAL(5,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 cast(1 as smallint) <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(10,0)) <=> cast(1 as decimal(10,0))) AS (CAST(1 AS SMALLINT) <=> 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 cast(1 as smallint) <=> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(20,0)) <=> cast(1 as decimal(20,0))) AS (CAST(1 AS SMALLINT) <=> CAST(1 AS DECIMAL(20,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 cast(1 as int) <=> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) <=> cast(cast(1 as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS INT) <=> CAST(1 AS DECIMAL(3,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 cast(1 as int) <=> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) <=> cast(cast(1 as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS INT) <=> CAST(1 AS DECIMAL(5,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 cast(1 as int) <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) <=> cast(1 as decimal(10,0))) AS (CAST(1 AS INT) <=> 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 cast(1 as int) <=> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as int) as decimal(10,0)) as decimal(20,0)) <=> cast(1 as decimal(20,0))) AS (CAST(1 AS INT) <=> CAST(1 AS DECIMAL(20,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 cast(1 as bigint) <=> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) <=> cast(cast(1 as decimal(3,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) <=> CAST(1 AS DECIMAL(3,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 cast(1 as bigint) <=> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) <=> cast(cast(1 as decimal(5,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) <=> CAST(1 AS DECIMAL(5,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 cast(1 as bigint) <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) <=> cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) <=> 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 cast(1 as bigint) <=> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) <=> cast(1 as decimal(20,0))) AS (CAST(1 AS BIGINT) <=> CAST(1 AS DECIMAL(20,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 cast(1 as float) <=> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) <=> cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS FLOAT) <=> CAST(1 AS DECIMAL(3,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 cast(1 as float) <=> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) <=> cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS FLOAT) <=> CAST(1 AS DECIMAL(5,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 cast(1 as float) <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) <=> cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS FLOAT) <=> 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 cast(1 as float) <=> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) <=> cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS FLOAT) <=> CAST(1 AS DECIMAL(20,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 cast(1 as double) <=> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) <=> cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS DOUBLE) <=> CAST(1 AS DECIMAL(3,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 cast(1 as double) <=> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) <=> cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS DOUBLE) <=> CAST(1 AS DECIMAL(5,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 cast(1 as double) <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) <=> cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS DOUBLE) <=> 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 cast(1 as double) <=> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) <=> cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS DOUBLE) <=> CAST(1 AS DECIMAL(20,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 cast(1 as decimal(10, 0)) <=> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) <=> cast(cast(1 as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(3,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 cast(1 as decimal(10, 0)) <=> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) <=> cast(cast(1 as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(5,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 cast(1 as decimal(10, 0)) <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) <=> cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <=> 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 cast(1 as decimal(10, 0)) <=> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) <=> cast(1 as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(20,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 cast('1' as binary) <=> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <=> cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <=> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <=> cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('1' as binary) <=> cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <=> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('1' as binary) <=> cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) <=> cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) <=> 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 + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) <=> cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) <=> 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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) <=> cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <=> 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 + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) <=> cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) <=> 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 + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(5,0)) <=> cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(3,0)) <=> 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 cast(1 as decimal(5, 0)) <=> cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) <=> cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) <=> 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 cast(1 as decimal(10, 0)) <=> cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) <=> cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <=> 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 cast(1 as decimal(20, 0)) <=> cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) <=> cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) <=> 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 cast(1 as decimal(3, 0)) <=> cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(10,0)) <=> cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) <=> 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 cast(1 as decimal(5, 0)) <=> cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(10,0)) <=> cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) <=> 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 cast(1 as decimal(10, 0)) <=> cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) <=> cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <=> 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 cast(1 as decimal(20, 0)) <=> cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) <=> cast(cast(cast(1 as int) as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) <=> 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 cast(1 as decimal(3, 0)) <=> cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(20,0)) <=> cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(3,0)) <=> 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 cast(1 as decimal(5, 0)) <=> cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(20,0)) <=> cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(5,0)) <=> 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 cast(1 as decimal(10, 0)) <=> cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) <=> cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) <=> 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 cast(1 as decimal(20, 0)) <=> cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) <=> cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) <=> 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 cast(1 as decimal(3, 0)) <=> cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) <=> cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(3,0)) <=> 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 cast(1 as decimal(5, 0)) <=> cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) <=> cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(5,0)) <=> 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 cast(1 as decimal(10, 0)) <=> cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) <=> cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(10,0)) <=> 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 cast(1 as decimal(20, 0)) <=> cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) <=> cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(20,0)) <=> 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 cast(1 as decimal(3, 0)) <=> cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) <=> cast(1 as double)) AS (CAST(1 AS DECIMAL(3,0)) <=> 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 cast(1 as decimal(5, 0)) <=> cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) <=> cast(1 as double)) AS (CAST(1 AS DECIMAL(5,0)) <=> 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 cast(1 as decimal(10, 0)) <=> cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) <=> cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) <=> 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 cast(1 as decimal(20, 0)) <=> cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) <=> cast(1 as double)) AS (CAST(1 AS DECIMAL(20,0)) <=> 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 cast(1 as decimal(3, 0)) <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(10,0)) <=> cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) <=> 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 cast(1 as decimal(5, 0)) <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(10,0)) <=> cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) <=> 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 cast(1 as decimal(10, 0)) <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) <=> cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <=> 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 cast(1 as decimal(20, 0)) <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) <=> cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) <=> 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 cast(1 as decimal(3, 0)) <=> cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast(1 as decimal(3, 0)) <=> cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast(1 as decimal(5, 0)) <=> cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast(1 as decimal(10, 0)) <=> cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast(1 as decimal(20, 0)) <=> cast('1' as binary)" + } ] +} + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) < cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) < cast(1 as decimal(3,0))) AS (CAST(1 AS TINYINT) < CAST(1 AS DECIMAL(3,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 cast(1 as tinyint) < cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(5,0)) < cast(1 as decimal(5,0))) AS (CAST(1 AS TINYINT) < CAST(1 AS DECIMAL(5,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 cast(1 as tinyint) < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(10,0)) < cast(1 as decimal(10,0))) AS (CAST(1 AS TINYINT) < 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 cast(1 as tinyint) < cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(20,0)) < cast(1 as decimal(20,0))) AS (CAST(1 AS TINYINT) < CAST(1 AS DECIMAL(20,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 cast(1 as smallint) < cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) < cast(cast(1 as decimal(3,0)) as decimal(5,0))) AS (CAST(1 AS SMALLINT) < CAST(1 AS DECIMAL(3,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 cast(1 as smallint) < cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) < cast(1 as decimal(5,0))) AS (CAST(1 AS SMALLINT) < CAST(1 AS DECIMAL(5,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 cast(1 as smallint) < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(10,0)) < cast(1 as decimal(10,0))) AS (CAST(1 AS SMALLINT) < 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 cast(1 as smallint) < cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(20,0)) < cast(1 as decimal(20,0))) AS (CAST(1 AS SMALLINT) < CAST(1 AS DECIMAL(20,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 cast(1 as int) < cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) < cast(cast(1 as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS INT) < CAST(1 AS DECIMAL(3,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 cast(1 as int) < cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) < cast(cast(1 as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS INT) < CAST(1 AS DECIMAL(5,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 cast(1 as int) < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) < cast(1 as decimal(10,0))) AS (CAST(1 AS INT) < 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 cast(1 as int) < cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as int) as decimal(10,0)) as decimal(20,0)) < cast(1 as decimal(20,0))) AS (CAST(1 AS INT) < CAST(1 AS DECIMAL(20,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 cast(1 as bigint) < cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) < cast(cast(1 as decimal(3,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) < CAST(1 AS DECIMAL(3,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 cast(1 as bigint) < cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) < cast(cast(1 as decimal(5,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) < CAST(1 AS DECIMAL(5,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 cast(1 as bigint) < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) < cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) < 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 cast(1 as bigint) < cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) < cast(1 as decimal(20,0))) AS (CAST(1 AS BIGINT) < CAST(1 AS DECIMAL(20,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 cast(1 as float) < cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) < cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS FLOAT) < CAST(1 AS DECIMAL(3,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 cast(1 as float) < cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) < cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS FLOAT) < CAST(1 AS DECIMAL(5,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 cast(1 as float) < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) < cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS FLOAT) < 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 cast(1 as float) < cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) < cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS FLOAT) < CAST(1 AS DECIMAL(20,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 cast(1 as double) < cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) < cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS DOUBLE) < CAST(1 AS DECIMAL(3,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 cast(1 as double) < cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) < cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS DOUBLE) < CAST(1 AS DECIMAL(5,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 cast(1 as double) < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) < cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS DOUBLE) < 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 cast(1 as double) < cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) < cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS DOUBLE) < CAST(1 AS DECIMAL(20,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 cast(1 as decimal(10, 0)) < cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) < cast(cast(1 as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) < CAST(1 AS DECIMAL(3,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 cast(1 as decimal(10, 0)) < cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) < cast(cast(1 as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) < CAST(1 AS DECIMAL(5,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 cast(1 as decimal(10, 0)) < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) < cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) < 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 cast(1 as decimal(10, 0)) < cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) < cast(1 as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) < CAST(1 AS DECIMAL(20,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 cast('1' as binary) < cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) < cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) < cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) < cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) < cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) < cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) < cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) < cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) < 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 + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) < cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) < 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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) < cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) < 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 + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) < cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) < 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 + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(5,0)) < cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(3,0)) < 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 cast(1 as decimal(5, 0)) < cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) < cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) < 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 cast(1 as decimal(10, 0)) < cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) < cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) < 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 cast(1 as decimal(20, 0)) < cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) < cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) < 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 cast(1 as decimal(3, 0)) < cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(10,0)) < cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) < 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 cast(1 as decimal(5, 0)) < cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(10,0)) < cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) < 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 cast(1 as decimal(10, 0)) < cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) < cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) < 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 cast(1 as decimal(20, 0)) < cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) < cast(cast(cast(1 as int) as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) < 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 cast(1 as decimal(3, 0)) < cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(20,0)) < cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(3,0)) < 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 cast(1 as decimal(5, 0)) < cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(20,0)) < cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(5,0)) < 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 cast(1 as decimal(10, 0)) < cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) < cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) < 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 cast(1 as decimal(20, 0)) < cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) < cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) < 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 cast(1 as decimal(3, 0)) < cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) < cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(3,0)) < 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 cast(1 as decimal(5, 0)) < cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) < cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(5,0)) < 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 cast(1 as decimal(10, 0)) < cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) < cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(10,0)) < 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 cast(1 as decimal(20, 0)) < cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) < cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(20,0)) < 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 cast(1 as decimal(3, 0)) < cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) < cast(1 as double)) AS (CAST(1 AS DECIMAL(3,0)) < 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 cast(1 as decimal(5, 0)) < cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) < cast(1 as double)) AS (CAST(1 AS DECIMAL(5,0)) < 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 cast(1 as decimal(10, 0)) < cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) < cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) < 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 cast(1 as decimal(20, 0)) < cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) < cast(1 as double)) AS (CAST(1 AS DECIMAL(20,0)) < 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 cast(1 as decimal(3, 0)) < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(10,0)) < cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) < 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 cast(1 as decimal(5, 0)) < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(10,0)) < cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) < 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 cast(1 as decimal(10, 0)) < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) < cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) < 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 cast(1 as decimal(20, 0)) < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) < cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) < 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 cast(1 as decimal(3, 0)) < cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) < cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) < cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) < cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) < cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) <= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) <= cast(1 as decimal(3,0))) AS (CAST(1 AS TINYINT) <= CAST(1 AS DECIMAL(3,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 cast(1 as tinyint) <= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(5,0)) <= cast(1 as decimal(5,0))) AS (CAST(1 AS TINYINT) <= CAST(1 AS DECIMAL(5,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 cast(1 as tinyint) <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(10,0)) <= cast(1 as decimal(10,0))) AS (CAST(1 AS TINYINT) <= 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 cast(1 as tinyint) <= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(20,0)) <= cast(1 as decimal(20,0))) AS (CAST(1 AS TINYINT) <= CAST(1 AS DECIMAL(20,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 cast(1 as smallint) <= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) <= cast(cast(1 as decimal(3,0)) as decimal(5,0))) AS (CAST(1 AS SMALLINT) <= CAST(1 AS DECIMAL(3,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 cast(1 as smallint) <= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) <= cast(1 as decimal(5,0))) AS (CAST(1 AS SMALLINT) <= CAST(1 AS DECIMAL(5,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 cast(1 as smallint) <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(10,0)) <= cast(1 as decimal(10,0))) AS (CAST(1 AS SMALLINT) <= 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 cast(1 as smallint) <= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(20,0)) <= cast(1 as decimal(20,0))) AS (CAST(1 AS SMALLINT) <= CAST(1 AS DECIMAL(20,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 cast(1 as int) <= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) <= cast(cast(1 as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS INT) <= CAST(1 AS DECIMAL(3,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 cast(1 as int) <= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) <= cast(cast(1 as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS INT) <= CAST(1 AS DECIMAL(5,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 cast(1 as int) <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) <= cast(1 as decimal(10,0))) AS (CAST(1 AS INT) <= 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 cast(1 as int) <= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as int) as decimal(10,0)) as decimal(20,0)) <= cast(1 as decimal(20,0))) AS (CAST(1 AS INT) <= CAST(1 AS DECIMAL(20,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 cast(1 as bigint) <= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) <= cast(cast(1 as decimal(3,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) <= CAST(1 AS DECIMAL(3,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 cast(1 as bigint) <= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) <= cast(cast(1 as decimal(5,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) <= CAST(1 AS DECIMAL(5,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 cast(1 as bigint) <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) <= cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) <= 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 cast(1 as bigint) <= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) <= cast(1 as decimal(20,0))) AS (CAST(1 AS BIGINT) <= CAST(1 AS DECIMAL(20,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 cast(1 as float) <= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) <= cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS FLOAT) <= CAST(1 AS DECIMAL(3,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 cast(1 as float) <= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) <= cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS FLOAT) <= CAST(1 AS DECIMAL(5,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 cast(1 as float) <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) <= cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS FLOAT) <= 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 cast(1 as float) <= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) <= cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS FLOAT) <= CAST(1 AS DECIMAL(20,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 cast(1 as double) <= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) <= cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS DOUBLE) <= CAST(1 AS DECIMAL(3,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 cast(1 as double) <= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) <= cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS DOUBLE) <= CAST(1 AS DECIMAL(5,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 cast(1 as double) <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) <= cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS DOUBLE) <= 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 cast(1 as double) <= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) <= cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS DOUBLE) <= CAST(1 AS DECIMAL(20,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 cast(1 as decimal(10, 0)) <= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) <= cast(cast(1 as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS DECIMAL(3,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 cast(1 as decimal(10, 0)) <= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) <= cast(cast(1 as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS DECIMAL(5,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 cast(1 as decimal(10, 0)) <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) <= cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <= 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 cast(1 as decimal(10, 0)) <= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) <= cast(1 as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS DECIMAL(20,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 cast('1' as binary) <= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) <= cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) <= cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <= cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <= cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) <= cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) <= 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 + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) <= cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) <= 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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) <= cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <= 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 + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) <= cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) <= 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 + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(5,0)) <= cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(3,0)) <= 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 cast(1 as decimal(5, 0)) <= cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) <= cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) <= 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 cast(1 as decimal(10, 0)) <= cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) <= cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <= 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 cast(1 as decimal(20, 0)) <= cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) <= cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) <= 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 cast(1 as decimal(3, 0)) <= cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(10,0)) <= cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) <= 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 cast(1 as decimal(5, 0)) <= cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(10,0)) <= cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) <= 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 cast(1 as decimal(10, 0)) <= cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) <= cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <= 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 cast(1 as decimal(20, 0)) <= cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) <= cast(cast(cast(1 as int) as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) <= 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 cast(1 as decimal(3, 0)) <= cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(20,0)) <= cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(3,0)) <= 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 cast(1 as decimal(5, 0)) <= cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(20,0)) <= cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(5,0)) <= 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 cast(1 as decimal(10, 0)) <= cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) <= cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) <= 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 cast(1 as decimal(20, 0)) <= cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) <= cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) <= 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 cast(1 as decimal(3, 0)) <= cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) <= cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(3,0)) <= 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 cast(1 as decimal(5, 0)) <= cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) <= cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(5,0)) <= 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 cast(1 as decimal(10, 0)) <= cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) <= cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(10,0)) <= 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 cast(1 as decimal(20, 0)) <= cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) <= cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(20,0)) <= 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 cast(1 as decimal(3, 0)) <= cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) <= cast(1 as double)) AS (CAST(1 AS DECIMAL(3,0)) <= 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 cast(1 as decimal(5, 0)) <= cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) <= cast(1 as double)) AS (CAST(1 AS DECIMAL(5,0)) <= 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 cast(1 as decimal(10, 0)) <= cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) <= cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) <= 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 cast(1 as decimal(20, 0)) <= cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) <= cast(1 as double)) AS (CAST(1 AS DECIMAL(20,0)) <= 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 cast(1 as decimal(3, 0)) <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(10,0)) <= cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) <= 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 cast(1 as decimal(5, 0)) <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(10,0)) <= cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) <= 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 cast(1 as decimal(10, 0)) <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) <= cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <= 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 cast(1 as decimal(20, 0)) <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) <= cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) <= 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 cast(1 as decimal(3, 0)) <= cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(3, 0)) <= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(5, 0)) <= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(10, 0)) <= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(20, 0)) <= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) > cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) > cast(1 as decimal(3,0))) AS (CAST(1 AS TINYINT) > CAST(1 AS DECIMAL(3,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 cast(1 as tinyint) > cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(5,0)) > cast(1 as decimal(5,0))) AS (CAST(1 AS TINYINT) > CAST(1 AS DECIMAL(5,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 cast(1 as tinyint) > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(10,0)) > cast(1 as decimal(10,0))) AS (CAST(1 AS TINYINT) > 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 cast(1 as tinyint) > cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(20,0)) > cast(1 as decimal(20,0))) AS (CAST(1 AS TINYINT) > CAST(1 AS DECIMAL(20,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 cast(1 as smallint) > cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) > cast(cast(1 as decimal(3,0)) as decimal(5,0))) AS (CAST(1 AS SMALLINT) > CAST(1 AS DECIMAL(3,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 cast(1 as smallint) > cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) > cast(1 as decimal(5,0))) AS (CAST(1 AS SMALLINT) > CAST(1 AS DECIMAL(5,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 cast(1 as smallint) > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(10,0)) > cast(1 as decimal(10,0))) AS (CAST(1 AS SMALLINT) > 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 cast(1 as smallint) > cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(20,0)) > cast(1 as decimal(20,0))) AS (CAST(1 AS SMALLINT) > CAST(1 AS DECIMAL(20,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 cast(1 as int) > cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) > cast(cast(1 as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS INT) > CAST(1 AS DECIMAL(3,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 cast(1 as int) > cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) > cast(cast(1 as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS INT) > CAST(1 AS DECIMAL(5,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 cast(1 as int) > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) > cast(1 as decimal(10,0))) AS (CAST(1 AS INT) > 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 cast(1 as int) > cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as int) as decimal(10,0)) as decimal(20,0)) > cast(1 as decimal(20,0))) AS (CAST(1 AS INT) > CAST(1 AS DECIMAL(20,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 cast(1 as bigint) > cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) > cast(cast(1 as decimal(3,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) > CAST(1 AS DECIMAL(3,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 cast(1 as bigint) > cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) > cast(cast(1 as decimal(5,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) > CAST(1 AS DECIMAL(5,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 cast(1 as bigint) > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) > cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) > 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 cast(1 as bigint) > cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) > cast(1 as decimal(20,0))) AS (CAST(1 AS BIGINT) > CAST(1 AS DECIMAL(20,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 cast(1 as float) > cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) > cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS FLOAT) > CAST(1 AS DECIMAL(3,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 cast(1 as float) > cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) > cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS FLOAT) > CAST(1 AS DECIMAL(5,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 cast(1 as float) > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) > cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS FLOAT) > 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 cast(1 as float) > cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) > cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS FLOAT) > CAST(1 AS DECIMAL(20,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 cast(1 as double) > cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) > cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS DOUBLE) > CAST(1 AS DECIMAL(3,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 cast(1 as double) > cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) > cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS DOUBLE) > CAST(1 AS DECIMAL(5,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 cast(1 as double) > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) > cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS DOUBLE) > 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 cast(1 as double) > cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) > cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS DOUBLE) > CAST(1 AS DECIMAL(20,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 cast(1 as decimal(10, 0)) > cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) > cast(cast(1 as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) > CAST(1 AS DECIMAL(3,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 cast(1 as decimal(10, 0)) > cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) > cast(cast(1 as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) > CAST(1 AS DECIMAL(5,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 cast(1 as decimal(10, 0)) > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) > cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) > 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 cast(1 as decimal(10, 0)) > cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) > cast(1 as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) > CAST(1 AS DECIMAL(20,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 cast('1' as binary) > cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) > cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) > cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) > cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) > cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) > cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) > cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) > cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) > 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 + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) > cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) > 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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) > cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) > 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 + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) > cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) > 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 + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(5,0)) > cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(3,0)) > 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 cast(1 as decimal(5, 0)) > cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) > cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) > 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 cast(1 as decimal(10, 0)) > cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) > cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) > 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 cast(1 as decimal(20, 0)) > cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) > cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) > 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 cast(1 as decimal(3, 0)) > cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(10,0)) > cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) > 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 cast(1 as decimal(5, 0)) > cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(10,0)) > cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) > 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 cast(1 as decimal(10, 0)) > cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) > cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) > 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 cast(1 as decimal(20, 0)) > cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) > cast(cast(cast(1 as int) as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) > 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 cast(1 as decimal(3, 0)) > cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(20,0)) > cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(3,0)) > 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 cast(1 as decimal(5, 0)) > cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(20,0)) > cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(5,0)) > 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 cast(1 as decimal(10, 0)) > cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) > cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) > 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 cast(1 as decimal(20, 0)) > cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) > cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) > 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 cast(1 as decimal(3, 0)) > cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) > cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(3,0)) > 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 cast(1 as decimal(5, 0)) > cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) > cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(5,0)) > 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 cast(1 as decimal(10, 0)) > cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) > cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(10,0)) > 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 cast(1 as decimal(20, 0)) > cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) > cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(20,0)) > 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 cast(1 as decimal(3, 0)) > cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) > cast(1 as double)) AS (CAST(1 AS DECIMAL(3,0)) > 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 cast(1 as decimal(5, 0)) > cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) > cast(1 as double)) AS (CAST(1 AS DECIMAL(5,0)) > 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 cast(1 as decimal(10, 0)) > cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) > cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) > 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 cast(1 as decimal(20, 0)) > cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) > cast(1 as double)) AS (CAST(1 AS DECIMAL(20,0)) > 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 cast(1 as decimal(3, 0)) > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(10,0)) > cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) > 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 cast(1 as decimal(5, 0)) > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(10,0)) > cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) > 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 cast(1 as decimal(10, 0)) > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) > cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) > 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 cast(1 as decimal(20, 0)) > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) > cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) > 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 cast(1 as decimal(3, 0)) > cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) > cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) > cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) > cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) > cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) >= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) >= cast(1 as decimal(3,0))) AS (CAST(1 AS TINYINT) >= CAST(1 AS DECIMAL(3,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 cast(1 as tinyint) >= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(5,0)) >= cast(1 as decimal(5,0))) AS (CAST(1 AS TINYINT) >= CAST(1 AS DECIMAL(5,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 cast(1 as tinyint) >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(10,0)) >= cast(1 as decimal(10,0))) AS (CAST(1 AS TINYINT) >= 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 cast(1 as tinyint) >= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(20,0)) >= cast(1 as decimal(20,0))) AS (CAST(1 AS TINYINT) >= CAST(1 AS DECIMAL(20,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 cast(1 as smallint) >= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) >= cast(cast(1 as decimal(3,0)) as decimal(5,0))) AS (CAST(1 AS SMALLINT) >= CAST(1 AS DECIMAL(3,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 cast(1 as smallint) >= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) >= cast(1 as decimal(5,0))) AS (CAST(1 AS SMALLINT) >= CAST(1 AS DECIMAL(5,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 cast(1 as smallint) >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(10,0)) >= cast(1 as decimal(10,0))) AS (CAST(1 AS SMALLINT) >= 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 cast(1 as smallint) >= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(20,0)) >= cast(1 as decimal(20,0))) AS (CAST(1 AS SMALLINT) >= CAST(1 AS DECIMAL(20,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 cast(1 as int) >= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) >= cast(cast(1 as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS INT) >= CAST(1 AS DECIMAL(3,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 cast(1 as int) >= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) >= cast(cast(1 as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS INT) >= CAST(1 AS DECIMAL(5,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 cast(1 as int) >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) >= cast(1 as decimal(10,0))) AS (CAST(1 AS INT) >= 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 cast(1 as int) >= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(cast(1 as int) as decimal(10,0)) as decimal(20,0)) >= cast(1 as decimal(20,0))) AS (CAST(1 AS INT) >= CAST(1 AS DECIMAL(20,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 cast(1 as bigint) >= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) >= cast(cast(1 as decimal(3,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) >= CAST(1 AS DECIMAL(3,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 cast(1 as bigint) >= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) >= cast(cast(1 as decimal(5,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) >= CAST(1 AS DECIMAL(5,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 cast(1 as bigint) >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) >= cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) >= 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 cast(1 as bigint) >= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) >= cast(1 as decimal(20,0))) AS (CAST(1 AS BIGINT) >= CAST(1 AS DECIMAL(20,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 cast(1 as float) >= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) >= cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS FLOAT) >= CAST(1 AS DECIMAL(3,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 cast(1 as float) >= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) >= cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS FLOAT) >= CAST(1 AS DECIMAL(5,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 cast(1 as float) >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) >= cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS FLOAT) >= 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 cast(1 as float) >= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) >= cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS FLOAT) >= CAST(1 AS DECIMAL(20,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 cast(1 as double) >= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) >= cast(cast(1 as decimal(3,0)) as double)) AS (CAST(1 AS DOUBLE) >= CAST(1 AS DECIMAL(3,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 cast(1 as double) >= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) >= cast(cast(1 as decimal(5,0)) as double)) AS (CAST(1 AS DOUBLE) >= CAST(1 AS DECIMAL(5,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 cast(1 as double) >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) >= cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS DOUBLE) >= 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 cast(1 as double) >= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) >= cast(cast(1 as decimal(20,0)) as double)) AS (CAST(1 AS DOUBLE) >= CAST(1 AS DECIMAL(20,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 cast(1 as decimal(10, 0)) >= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) >= cast(cast(1 as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS DECIMAL(3,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 cast(1 as decimal(10, 0)) >= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) >= cast(cast(1 as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS DECIMAL(5,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 cast(1 as decimal(10, 0)) >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) >= cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) >= 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 cast(1 as decimal(10, 0)) >= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) >= cast(1 as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS DECIMAL(20,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 cast('1' as binary) >= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) >= cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) >= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) >= cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) >= cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) >= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) >= cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(3,0)) >= cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) >= 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 + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) >= cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) >= 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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) >= cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) >= 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 + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) >= cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) >= 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 + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(5,0)) >= cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(3,0)) >= 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 cast(1 as decimal(5, 0)) >= cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(5,0)) >= cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) >= 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 cast(1 as decimal(10, 0)) >= cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) >= cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) >= 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 cast(1 as decimal(20, 0)) >= cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) >= cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) >= 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 cast(1 as decimal(3, 0)) >= cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(10,0)) >= cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) >= 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 cast(1 as decimal(5, 0)) >= cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(10,0)) >= cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) >= 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 cast(1 as decimal(10, 0)) >= cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) >= cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) >= 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 cast(1 as decimal(20, 0)) >= cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) >= cast(cast(cast(1 as int) as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) >= 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 cast(1 as decimal(3, 0)) >= cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(20,0)) >= cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(3,0)) >= 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 cast(1 as decimal(5, 0)) >= cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(20,0)) >= cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(5,0)) >= 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 cast(1 as decimal(10, 0)) >= cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) >= cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) >= 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 cast(1 as decimal(20, 0)) >= cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) >= cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) >= 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 cast(1 as decimal(3, 0)) >= cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) >= cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(3,0)) >= 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 cast(1 as decimal(5, 0)) >= cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) >= cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(5,0)) >= 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 cast(1 as decimal(10, 0)) >= cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) >= cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(10,0)) >= 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 cast(1 as decimal(20, 0)) >= cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) >= cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(20,0)) >= 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 cast(1 as decimal(3, 0)) >= cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as double) >= cast(1 as double)) AS (CAST(1 AS DECIMAL(3,0)) >= 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 cast(1 as decimal(5, 0)) >= cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as double) >= cast(1 as double)) AS (CAST(1 AS DECIMAL(5,0)) >= 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 cast(1 as decimal(10, 0)) >= cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) >= cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) >= 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 cast(1 as decimal(20, 0)) >= cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(20,0)) as double) >= cast(1 as double)) AS (CAST(1 AS DECIMAL(20,0)) >= 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 cast(1 as decimal(3, 0)) >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(3,0)) as decimal(10,0)) >= cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(3,0)) >= 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 cast(1 as decimal(5, 0)) >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(5,0)) as decimal(10,0)) >= cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(5,0)) >= 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 cast(1 as decimal(10, 0)) >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) >= cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) >= 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 cast(1 as decimal(20, 0)) >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(20,0)) >= cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) >= 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 cast(1 as decimal(3, 0)) >= cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(3, 0)) >= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(5, 0)) >= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(10, 0)) >= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(20, 0)) >= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +org.apache.spark.sql.AnalysisException +{ + "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 +SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) <> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as tinyint) as decimal(3,0)) = cast(1 as decimal(3,0))) AS (NOT (CAST(1 AS TINYINT) = CAST(1 AS DECIMAL(3,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 cast(1 as tinyint) <> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(5,0)) = cast(1 as decimal(5,0))) AS (NOT (CAST(1 AS TINYINT) = CAST(1 AS DECIMAL(5,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 cast(1 as tinyint) <> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(10,0)) = cast(1 as decimal(10,0))) AS (NOT (CAST(1 AS TINYINT) = 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 cast(1 as tinyint) <> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(20,0)) = cast(1 as decimal(20,0))) AS (NOT (CAST(1 AS TINYINT) = CAST(1 AS DECIMAL(20,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 cast(1 as smallint) <> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as smallint) as decimal(5,0)) = cast(cast(1 as decimal(3,0)) as decimal(5,0))) AS (NOT (CAST(1 AS SMALLINT) = CAST(1 AS DECIMAL(3,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 cast(1 as smallint) <> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as smallint) as decimal(5,0)) = cast(1 as decimal(5,0))) AS (NOT (CAST(1 AS SMALLINT) = CAST(1 AS DECIMAL(5,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 cast(1 as smallint) <> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(10,0)) = cast(1 as decimal(10,0))) AS (NOT (CAST(1 AS SMALLINT) = 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 cast(1 as smallint) <> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(20,0)) = cast(1 as decimal(20,0))) AS (NOT (CAST(1 AS SMALLINT) = CAST(1 AS DECIMAL(20,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 cast(1 as int) <> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as int) as decimal(10,0)) = cast(cast(1 as decimal(3,0)) as decimal(10,0))) AS (NOT (CAST(1 AS INT) = CAST(1 AS DECIMAL(3,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 cast(1 as int) <> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as int) as decimal(10,0)) = cast(cast(1 as decimal(5,0)) as decimal(10,0))) AS (NOT (CAST(1 AS INT) = CAST(1 AS DECIMAL(5,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 cast(1 as int) <> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as int) as decimal(10,0)) = cast(1 as decimal(10,0))) AS (NOT (CAST(1 AS INT) = 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 cast(1 as int) <> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(cast(1 as int) as decimal(10,0)) as decimal(20,0)) = cast(1 as decimal(20,0))) AS (NOT (CAST(1 AS INT) = CAST(1 AS DECIMAL(20,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 cast(1 as bigint) <> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as bigint) as decimal(20,0)) = cast(cast(1 as decimal(3,0)) as decimal(20,0))) AS (NOT (CAST(1 AS BIGINT) = CAST(1 AS DECIMAL(3,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 cast(1 as bigint) <> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as bigint) as decimal(20,0)) = cast(cast(1 as decimal(5,0)) as decimal(20,0))) AS (NOT (CAST(1 AS BIGINT) = CAST(1 AS DECIMAL(5,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 cast(1 as bigint) <> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as bigint) as decimal(20,0)) = cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (NOT (CAST(1 AS BIGINT) = 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 cast(1 as bigint) <> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as bigint) as decimal(20,0)) = cast(1 as decimal(20,0))) AS (NOT (CAST(1 AS BIGINT) = CAST(1 AS DECIMAL(20,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 cast(1 as float) <> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as float) as double) = cast(cast(1 as decimal(3,0)) as double)) AS (NOT (CAST(1 AS FLOAT) = CAST(1 AS DECIMAL(3,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 cast(1 as float) <> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as float) as double) = cast(cast(1 as decimal(5,0)) as double)) AS (NOT (CAST(1 AS FLOAT) = CAST(1 AS DECIMAL(5,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 cast(1 as float) <> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as float) as double) = cast(cast(1 as decimal(10,0)) as double)) AS (NOT (CAST(1 AS FLOAT) = 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 cast(1 as float) <> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as float) as double) = cast(cast(1 as decimal(20,0)) as double)) AS (NOT (CAST(1 AS FLOAT) = CAST(1 AS DECIMAL(20,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 cast(1 as double) <> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [NOT (cast(1 as double) = cast(cast(1 as decimal(3,0)) as double)) AS (NOT (CAST(1 AS DOUBLE) = CAST(1 AS DECIMAL(3,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 cast(1 as double) <> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [NOT (cast(1 as double) = cast(cast(1 as decimal(5,0)) as double)) AS (NOT (CAST(1 AS DOUBLE) = CAST(1 AS DECIMAL(5,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 cast(1 as double) <> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [NOT (cast(1 as double) = cast(cast(1 as decimal(10,0)) as double)) AS (NOT (CAST(1 AS DOUBLE) = 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 cast(1 as double) <> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [NOT (cast(1 as double) = cast(cast(1 as decimal(20,0)) as double)) AS (NOT (CAST(1 AS DOUBLE) = CAST(1 AS DECIMAL(20,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 cast(1 as decimal(10, 0)) <> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +Project [NOT (cast(1 as decimal(10,0)) = cast(cast(1 as decimal(3,0)) as decimal(10,0))) AS (NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(3,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 cast(1 as decimal(10, 0)) <> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +Project [NOT (cast(1 as decimal(10,0)) = cast(cast(1 as decimal(5,0)) as decimal(10,0))) AS (NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(5,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 cast(1 as decimal(10, 0)) <> 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 (CAST(1 AS DECIMAL(10,0)) = 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 cast(1 as decimal(10, 0)) <> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(10,0)) as decimal(20,0)) = cast(1 as decimal(20,0))) AS (NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(20,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 cast('1' as binary) <> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) <> cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) <> cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <> cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <> cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(3, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(5, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(20, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast(1 as tinyint) FROM t +-- !query analysis +Project [NOT (cast(1 as decimal(3,0)) = cast(cast(1 as tinyint) as decimal(3,0))) AS (NOT (CAST(1 AS DECIMAL(3,0)) = 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 + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast(1 as tinyint) FROM t +-- !query analysis +Project [NOT (cast(1 as decimal(5,0)) = cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(5,0))) AS (NOT (CAST(1 AS DECIMAL(5,0)) = 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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(1 as tinyint) FROM t +-- !query analysis +Project [NOT (cast(1 as decimal(10,0)) = cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(10,0))) AS (NOT (CAST(1 AS DECIMAL(10,0)) = 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 + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast(1 as tinyint) FROM t +-- !query analysis +Project [NOT (cast(1 as decimal(20,0)) = cast(cast(cast(1 as tinyint) as decimal(3,0)) as decimal(20,0))) AS (NOT (CAST(1 AS DECIMAL(20,0)) = 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 + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast(1 as smallint) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(3,0)) as decimal(5,0)) = cast(cast(1 as smallint) as decimal(5,0))) AS (NOT (CAST(1 AS DECIMAL(3,0)) = 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 cast(1 as decimal(5, 0)) <> cast(1 as smallint) FROM t +-- !query analysis +Project [NOT (cast(1 as decimal(5,0)) = cast(cast(1 as smallint) as decimal(5,0))) AS (NOT (CAST(1 AS DECIMAL(5,0)) = 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 cast(1 as decimal(10, 0)) <> cast(1 as smallint) FROM t +-- !query analysis +Project [NOT (cast(1 as decimal(10,0)) = cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(10,0))) AS (NOT (CAST(1 AS DECIMAL(10,0)) = 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 cast(1 as decimal(20, 0)) <> cast(1 as smallint) FROM t +-- !query analysis +Project [NOT (cast(1 as decimal(20,0)) = cast(cast(cast(1 as smallint) as decimal(5,0)) as decimal(20,0))) AS (NOT (CAST(1 AS DECIMAL(20,0)) = 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 cast(1 as decimal(3, 0)) <> cast(1 as int) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(3,0)) as decimal(10,0)) = cast(cast(1 as int) as decimal(10,0))) AS (NOT (CAST(1 AS DECIMAL(3,0)) = 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 cast(1 as decimal(5, 0)) <> cast(1 as int) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(5,0)) as decimal(10,0)) = cast(cast(1 as int) as decimal(10,0))) AS (NOT (CAST(1 AS DECIMAL(5,0)) = 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 cast(1 as decimal(10, 0)) <> cast(1 as int) FROM t +-- !query analysis +Project [NOT (cast(1 as decimal(10,0)) = cast(cast(1 as int) as decimal(10,0))) AS (NOT (CAST(1 AS DECIMAL(10,0)) = 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 cast(1 as decimal(20, 0)) <> cast(1 as int) FROM t +-- !query analysis +Project [NOT (cast(1 as decimal(20,0)) = cast(cast(cast(1 as int) as decimal(10,0)) as decimal(20,0))) AS (NOT (CAST(1 AS DECIMAL(20,0)) = 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 cast(1 as decimal(3, 0)) <> cast(1 as bigint) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(3,0)) as decimal(20,0)) = cast(cast(1 as bigint) as decimal(20,0))) AS (NOT (CAST(1 AS DECIMAL(3,0)) = 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 cast(1 as decimal(5, 0)) <> cast(1 as bigint) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(5,0)) as decimal(20,0)) = cast(cast(1 as bigint) as decimal(20,0))) AS (NOT (CAST(1 AS DECIMAL(5,0)) = 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 cast(1 as decimal(10, 0)) <> cast(1 as bigint) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(10,0)) as decimal(20,0)) = cast(cast(1 as bigint) as decimal(20,0))) AS (NOT (CAST(1 AS DECIMAL(10,0)) = 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 cast(1 as decimal(20, 0)) <> cast(1 as bigint) FROM t +-- !query analysis +Project [NOT (cast(1 as decimal(20,0)) = cast(cast(1 as bigint) as decimal(20,0))) AS (NOT (CAST(1 AS DECIMAL(20,0)) = 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 cast(1 as decimal(3, 0)) <> cast(1 as float) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(3,0)) as double) = cast(cast(1 as float) as double)) AS (NOT (CAST(1 AS DECIMAL(3,0)) = 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 cast(1 as decimal(5, 0)) <> cast(1 as float) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(5,0)) as double) = cast(cast(1 as float) as double)) AS (NOT (CAST(1 AS DECIMAL(5,0)) = 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 cast(1 as decimal(10, 0)) <> cast(1 as float) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(10,0)) as double) = cast(cast(1 as float) as double)) AS (NOT (CAST(1 AS DECIMAL(10,0)) = 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 cast(1 as decimal(20, 0)) <> cast(1 as float) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(20,0)) as double) = cast(cast(1 as float) as double)) AS (NOT (CAST(1 AS DECIMAL(20,0)) = 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 cast(1 as decimal(3, 0)) <> cast(1 as double) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(3,0)) as double) = cast(1 as double)) AS (NOT (CAST(1 AS DECIMAL(3,0)) = 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 cast(1 as decimal(5, 0)) <> cast(1 as double) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(5,0)) as double) = cast(1 as double)) AS (NOT (CAST(1 AS DECIMAL(5,0)) = 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 cast(1 as decimal(10, 0)) <> cast(1 as double) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(10,0)) as double) = cast(1 as double)) AS (NOT (CAST(1 AS DECIMAL(10,0)) = 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 cast(1 as decimal(20, 0)) <> cast(1 as double) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(20,0)) as double) = cast(1 as double)) AS (NOT (CAST(1 AS DECIMAL(20,0)) = 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 cast(1 as decimal(3, 0)) <> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(3,0)) as decimal(10,0)) = cast(1 as decimal(10,0))) AS (NOT (CAST(1 AS DECIMAL(3,0)) = 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 cast(1 as decimal(5, 0)) <> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [NOT (cast(cast(1 as decimal(5,0)) as decimal(10,0)) = cast(1 as decimal(10,0))) AS (NOT (CAST(1 AS DECIMAL(5,0)) = 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 cast(1 as decimal(10, 0)) <> 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 (CAST(1 AS DECIMAL(10,0)) = 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 cast(1 as decimal(20, 0)) <> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [NOT (cast(1 as decimal(20,0)) = cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (NOT (CAST(1 AS DECIMAL(20,0)) = 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 cast(1 as decimal(3, 0)) <> cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(3, 0)) <> cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(5, 0)) <> cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(10, 0)) <> cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(20, 0)) <> cast('1' as binary)" + } ] +} + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !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 + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00' as date)" + } ] +} 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 new file mode 100644 index 0000000000000..671c5c550cc65 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/division.sql.out @@ -0,0 +1,2471 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query analysis +CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as double) / cast(cast(1 as tinyint) as double)) AS (CAST(1 AS TINYINT) / 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 + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as double) / cast(cast(1 as smallint) as double)) AS (CAST(1 AS TINYINT) / 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 cast(1 as tinyint) / cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as double) / cast(cast(1 as int) as double)) AS (CAST(1 AS TINYINT) / 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 cast(1 as tinyint) / cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as double) / cast(cast(1 as bigint) as double)) AS (CAST(1 AS TINYINT) / 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 cast(1 as tinyint) / cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as double) / cast(cast(1 as float) as double)) AS (CAST(1 AS TINYINT) / 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 cast(1 as tinyint) / cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as double) / cast(cast(1 as double) as double)) AS (CAST(1 AS TINYINT) / 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 cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as tinyint) as decimal(3,0)) / cast(1 as decimal(10,0))) AS (CAST(1 AS TINYINT) / 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 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] ++- 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) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast(1 as tinyint) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as tinyint) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as tinyint) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "cast(1 as tinyint) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as smallint) / cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as double) / cast(cast(1 as tinyint) as double)) AS (CAST(1 AS SMALLINT) / 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 + + +-- !query +SELECT cast(1 as smallint) / cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as double) / cast(cast(1 as smallint) as double)) AS (CAST(1 AS SMALLINT) / 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 cast(1 as smallint) / cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as double) / cast(cast(1 as int) as double)) AS (CAST(1 AS SMALLINT) / 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 cast(1 as smallint) / cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as double) / cast(cast(1 as bigint) as double)) AS (CAST(1 AS SMALLINT) / 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 cast(1 as smallint) / cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as double) / cast(cast(1 as float) as double)) AS (CAST(1 AS SMALLINT) / 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 cast(1 as smallint) / cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as double) / cast(cast(1 as double) as double)) AS (CAST(1 AS SMALLINT) / 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 cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as smallint) as decimal(5,0)) / cast(1 as decimal(10,0))) AS (CAST(1 AS SMALLINT) / 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 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] ++- 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 smallint) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "cast(1 as smallint) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as smallint) / cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast(1 as smallint) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as smallint) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "cast(1 as smallint) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as int) / cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as double) / cast(cast(1 as tinyint) as double)) AS (CAST(1 AS INT) / 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 + + +-- !query +SELECT cast(1 as int) / cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as double) / cast(cast(1 as smallint) as double)) AS (CAST(1 AS INT) / 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 cast(1 as int) / cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as double) / cast(cast(1 as int) as double)) AS (CAST(1 AS INT) / 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 cast(1 as int) / cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as double) / cast(cast(1 as bigint) as double)) AS (CAST(1 AS INT) / 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 cast(1 as int) / cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as double) / cast(cast(1 as float) as double)) AS (CAST(1 AS INT) / 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 cast(1 as int) / cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as double) / cast(cast(1 as double) as double)) AS (CAST(1 AS INT) / 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 cast(1 as int) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as int) as decimal(10,0)) / cast(1 as decimal(10,0))) AS (CAST(1 AS INT) / 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 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] ++- 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 int) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS INT) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "cast(1 as int) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as int) / cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS INT) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(1 as int) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as int) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS INT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 66, + "fragment" : "cast(1 as int) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as int) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS INT) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "cast(1 as int) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as bigint) / cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as double) / cast(cast(1 as tinyint) as double)) AS (CAST(1 AS BIGINT) / 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 + + +-- !query +SELECT cast(1 as bigint) / cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as double) / cast(cast(1 as smallint) as double)) AS (CAST(1 AS BIGINT) / 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 cast(1 as bigint) / cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as double) / cast(cast(1 as int) as double)) AS (CAST(1 AS BIGINT) / 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 cast(1 as bigint) / cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as double) / cast(cast(1 as bigint) as double)) AS (CAST(1 AS BIGINT) / 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 cast(1 as bigint) / cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as double) / cast(cast(1 as float) as double)) AS (CAST(1 AS BIGINT) / 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 cast(1 as bigint) / cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as double) / cast(cast(1 as double) as double)) AS (CAST(1 AS BIGINT) / 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 cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as bigint) as decimal(20,0)) / cast(1 as decimal(10,0))) AS (CAST(1 AS BIGINT) / 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 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] ++- 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 bigint) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as bigint) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as bigint) / cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as bigint) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast(1 as bigint) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast(1 as bigint) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as float) / cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) / cast(cast(1 as tinyint) as double)) AS (CAST(1 AS FLOAT) / 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 + + +-- !query +SELECT cast(1 as float) / cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) / cast(cast(1 as smallint) as double)) AS (CAST(1 AS FLOAT) / 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 cast(1 as float) / cast(1 as int) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) / cast(cast(1 as int) as double)) AS (CAST(1 AS FLOAT) / 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 cast(1 as float) / cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) / cast(cast(1 as bigint) as double)) AS (CAST(1 AS FLOAT) / 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 cast(1 as float) / cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) / cast(cast(1 as float) as double)) AS (CAST(1 AS FLOAT) / 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 cast(1 as float) / cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) / cast(cast(1 as double) as double)) AS (CAST(1 AS FLOAT) / 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 cast(1 as float) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as float) as double) / cast(cast(cast(1 as decimal(10,0)) as double) as double)) AS (CAST(1 AS FLOAT) / 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 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] ++- 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 float) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as float) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as float) / cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "cast(1 as float) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as float) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 68, + "fragment" : "cast(1 as float) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as float) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "cast(1 as float) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as double) / cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as double) / cast(cast(1 as tinyint) as double)) AS (CAST(1 AS DOUBLE) / 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 + + +-- !query +SELECT cast(1 as double) / cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as double) / cast(cast(1 as smallint) as double)) AS (CAST(1 AS DOUBLE) / 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 cast(1 as double) / cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as double) / cast(cast(1 as int) as double)) AS (CAST(1 AS DOUBLE) / 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 cast(1 as double) / cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as double) / cast(cast(1 as bigint) as double)) AS (CAST(1 AS DOUBLE) / 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 cast(1 as double) / cast(1 as float) FROM t +-- !query analysis +Project [(cast(1 as double) / cast(cast(1 as float) as double)) AS (CAST(1 AS DOUBLE) / 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 cast(1 as double) / cast(1 as double) FROM t +-- !query analysis +Project [(cast(1 as double) / cast(1 as double)) AS (CAST(1 AS DOUBLE) / 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 cast(1 as double) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as double) / cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS DOUBLE) / 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 cast(1 as double) / cast(1 as string) FROM t +-- !query analysis +Project [(cast(1 as double) / cast(cast(1 as string) as double)) AS (CAST(1 AS DOUBLE) / 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 cast(1 as double) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as double) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as double) / cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as double) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as double) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast(1 as double) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as double) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast(1 as double) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) / cast(cast(1 as tinyint) as decimal(3,0))) AS (CAST(1 AS DECIMAL(10,0)) / 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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) / cast(cast(1 as smallint) as decimal(5,0))) AS (CAST(1 AS DECIMAL(10,0)) / 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 cast(1 as decimal(10, 0)) / cast(1 as int) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) / cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) / 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 cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) / cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) / 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 cast(1 as decimal(10, 0)) / cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) / cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(10,0)) / 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 cast(1 as decimal(10, 0)) / cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as decimal(10,0)) as double) / cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) / 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 cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(1 as decimal(10,0)) / cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) / 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 cast(1 as decimal(10, 0)) / cast(1 as string) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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(10, 0)) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !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] ++- 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) / 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] ++- 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) / 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] ++- 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) / 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] ++- 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) / cast(1 as float) FROM t +-- !query analysis +Project [(cast(cast(1 as string) as double) / cast(cast(1 as float) as double)) AS (CAST(1 AS STRING) / 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 cast(1 as string) / cast(1 as double) FROM t +-- !query analysis +Project [(cast(cast(1 as string) as double) / cast(1 as double)) AS (CAST(1 AS STRING) / 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 cast(1 as string) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +Project [(cast(cast(1 as string) as double) / cast(cast(1 as decimal(10,0)) as double)) AS (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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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 + + +-- !query +SELECT cast(1 as string) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as string) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as string) / cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as string) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as string) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast(1 as string) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as string) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast(1 as string) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as tinyint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast('1' as binary) / cast(1 as tinyint)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as smallint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "cast('1' as binary) / cast(1 as smallint)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as int) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "cast('1' as binary) / cast(1 as int)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as bigint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast('1' as binary) / cast(1 as bigint)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as float) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast('1' as binary) / cast(1 as float)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as double) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast('1' as binary) / cast(1 as double)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) / cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as string) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS STRING))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast('1' as binary) / cast(1 as string)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "cast('1' as binary) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast('1' as binary) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('1' as binary) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "cast('1' as binary) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as tinyint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as boolean) / cast(1 as tinyint)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as smallint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast(1 as boolean) / cast(1 as smallint)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as int) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(1 as boolean) / cast(1 as int)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as bigint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as boolean) / cast(1 as bigint)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as float) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "cast(1 as boolean) / cast(1 as float)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as double) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as boolean) / cast(1 as double)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as boolean) / cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as string) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS STRING))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as boolean) / cast(1 as string)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast(1 as boolean) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as boolean) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as boolean) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "cast(1 as boolean) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as tinyint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as tinyint)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as smallint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as smallint)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as int) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 66, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as int)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as bigint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as bigint)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as float) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 68, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as float)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as double) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as double)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as string) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS STRING))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as string)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 94, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 87, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as tinyint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as tinyint)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as smallint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as smallint)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as int) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as int)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as bigint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as bigint)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as float) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as float)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as double) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as double)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as string) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS STRING))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as string)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 87, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"DATE\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00' as date)" + } ] +} 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 new file mode 100644 index 0000000000000..5a9b5ddbafa39 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/elt.sql.out @@ -0,0 +1,76 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT elt(2, col1, col2, col3, col4, col5) col +FROM ( + SELECT + 'prefix_' col1, + id col2, + string(id + 1) col3, + encode(string(id + 2), 'utf-8') col4, + CAST(id AS DOUBLE) col5 + 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] ++- 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, splits=None) + + +-- !query +SELECT elt(3, col1, col2, col3, col4) col +FROM ( + SELECT + string(id) col1, + string(id + 1) col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + 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] ++- 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, splits=None) + + +-- !query +set spark.sql.function.eltOutputAsString=true +-- !query analysis +SetCommand (spark.sql.function.eltOutputAsString,Some(true)) + + +-- !query +SELECT elt(1, col1, col2) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2 + FROM range(10) +) +-- !query analysis +Project [elt(1, cast(col1#x as string), cast(col2#x as string), false) 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, splits=None) + + +-- !query +set spark.sql.function.eltOutputAsString=false +-- !query analysis +SetCommand (spark.sql.function.eltOutputAsString,Some(false)) + + +-- !query +SELECT elt(2, col1, col2) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2 + FROM range(10) +) +-- !query analysis +Project [elt(2, col1#x, col2#x, false) 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, splits=None) 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 new file mode 100644 index 0000000000000..c93a21f3f37c7 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/ifCoercion.sql.out @@ -0,0 +1,2361 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query analysis +CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast(2 as tinyint)) FROM t +-- !query analysis +Project [if (true) cast(1 as tinyint) else cast(2 as tinyint) AS (IF(true, CAST(1 AS TINYINT), CAST(2 AS TINYINT)))#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 tinyint), cast(2 as smallint)) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as tinyint) as smallint) else cast(2 as smallint) AS (IF(true, CAST(1 AS TINYINT), CAST(2 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 IF(true, cast(1 as tinyint), cast(2 as int)) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as tinyint) as int) else cast(2 as int) AS (IF(true, CAST(1 AS TINYINT), CAST(2 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 IF(true, cast(1 as tinyint), cast(2 as bigint)) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as tinyint) as bigint) else cast(2 as bigint) AS (IF(true, CAST(1 AS TINYINT), 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 +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] ++- 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 tinyint), cast(2 as double)) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as tinyint) as double) else cast(2 as double) AS (IF(true, CAST(1 AS TINYINT), 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 +SELECT IF(true, cast(1 as tinyint), cast(2 as decimal(10, 0))) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as tinyint) as decimal(10,0)) else cast(2 as decimal(10,0)) AS (IF(true, CAST(1 AS TINYINT), CAST(2 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 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] ++- 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 tinyint), cast('2' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS TINYINT), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "IF(true, cast(1 as tinyint), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast(2 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS TINYINT), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "IF(true, cast(1 as tinyint), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS TINYINT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS TINYINT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as smallint), cast(2 as tinyint)) FROM t +-- !query analysis +Project [if (true) cast(1 as smallint) else cast(cast(2 as tinyint) as smallint) AS (IF(true, CAST(1 AS SMALLINT), CAST(2 AS TINYINT)))#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 smallint), cast(2 as smallint)) FROM t +-- !query analysis +Project [if (true) cast(1 as smallint) else cast(2 as smallint) AS (IF(true, CAST(1 AS SMALLINT), CAST(2 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 IF(true, cast(1 as smallint), cast(2 as int)) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as smallint) as int) else cast(2 as int) AS (IF(true, CAST(1 AS SMALLINT), CAST(2 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 IF(true, cast(1 as smallint), cast(2 as bigint)) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as smallint) as bigint) else cast(2 as bigint) AS (IF(true, CAST(1 AS SMALLINT), 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 +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] ++- 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 smallint), cast(2 as double)) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as smallint) as double) else cast(2 as double) AS (IF(true, CAST(1 AS SMALLINT), 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 +SELECT IF(true, cast(1 as smallint), cast(2 as decimal(10, 0))) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as smallint) as decimal(10,0)) else cast(2 as decimal(10,0)) AS (IF(true, CAST(1 AS SMALLINT), CAST(2 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 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] ++- 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 smallint), cast('2' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS SMALLINT), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "IF(true, cast(1 as smallint), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as smallint), cast(2 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS SMALLINT), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "IF(true, cast(1 as smallint), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS SMALLINT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS SMALLINT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as int), cast(2 as tinyint)) FROM t +-- !query analysis +Project [if (true) cast(1 as int) else cast(cast(2 as tinyint) as int) AS (IF(true, CAST(1 AS INT), CAST(2 AS TINYINT)))#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 int), cast(2 as smallint)) FROM t +-- !query analysis +Project [if (true) cast(1 as int) else cast(cast(2 as smallint) as int) AS (IF(true, CAST(1 AS INT), CAST(2 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 IF(true, cast(1 as int), cast(2 as int)) FROM t +-- !query analysis +Project [if (true) cast(1 as int) else cast(2 as int) AS (IF(true, CAST(1 AS INT), CAST(2 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 IF(true, cast(1 as int), cast(2 as bigint)) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as int) as bigint) else cast(2 as bigint) AS (IF(true, CAST(1 AS INT), 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 +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] ++- 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 int), cast(2 as double)) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as int) as double) else cast(2 as double) AS (IF(true, CAST(1 AS INT), 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 +SELECT IF(true, cast(1 as int), cast(2 as decimal(10, 0))) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as int) as decimal(10,0)) else cast(2 as decimal(10,0)) AS (IF(true, CAST(1 AS INT), CAST(2 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 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] ++- 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 int), cast('2' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS INT), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "IF(true, cast(1 as int), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as int), cast(2 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS INT), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "IF(true, cast(1 as int), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as int), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS INT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "IF(true, cast(1 as int), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as int), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS INT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 68, + "fragment" : "IF(true, cast(1 as int), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as bigint), cast(2 as tinyint)) FROM t +-- !query analysis +Project [if (true) cast(1 as bigint) else cast(cast(2 as tinyint) as bigint) AS (IF(true, CAST(1 AS BIGINT), CAST(2 AS TINYINT)))#xL] ++- 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 bigint), cast(2 as smallint)) FROM t +-- !query analysis +Project [if (true) cast(1 as bigint) else cast(cast(2 as smallint) as bigint) AS (IF(true, CAST(1 AS BIGINT), CAST(2 AS SMALLINT)))#xL] ++- 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 bigint), cast(2 as int)) FROM t +-- !query analysis +Project [if (true) cast(1 as bigint) else cast(cast(2 as int) as bigint) AS (IF(true, CAST(1 AS BIGINT), CAST(2 AS INT)))#xL] ++- 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 bigint), cast(2 as bigint)) FROM t +-- !query analysis +Project [if (true) cast(1 as bigint) else cast(2 as bigint) AS (IF(true, CAST(1 AS BIGINT), 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 +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] ++- 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 bigint), cast(2 as double)) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as bigint) as double) else cast(2 as double) AS (IF(true, CAST(1 AS BIGINT), 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 +SELECT IF(true, cast(1 as bigint), cast(2 as decimal(10, 0))) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as bigint) as decimal(20,0)) else cast(cast(2 as decimal(10,0)) as decimal(20,0)) AS (IF(true, CAST(1 AS BIGINT), CAST(2 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 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] ++- 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 bigint), cast('2' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BIGINT), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "IF(true, cast(1 as bigint), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as bigint), cast(2 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BIGINT), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "IF(true, cast(1 as bigint), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BIGINT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BIGINT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 float), cast(2 as float)) FROM t +-- !query analysis +Project [if (true) cast(1 as float) else cast(2 as float) AS (IF(true, CAST(1 AS FLOAT), 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 +SELECT IF(true, cast(1 as float), cast(2 as double)) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as float) as double) else cast(2 as double) AS (IF(true, CAST(1 AS FLOAT), 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 +SELECT IF(true, cast(1 as float), cast(2 as decimal(10, 0))) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as float) as double) else cast(cast(2 as decimal(10,0)) as double) AS (IF(true, CAST(1 AS FLOAT), CAST(2 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 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] ++- 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 float), cast('2' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS FLOAT), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "IF(true, cast(1 as float), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as float), cast(2 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS FLOAT), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "IF(true, cast(1 as float), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as float), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS FLOAT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "IF(true, cast(1 as float), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as float), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS FLOAT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "IF(true, cast(1 as float), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as double), cast(2 as tinyint)) FROM t +-- !query analysis +Project [if (true) cast(1 as double) else cast(cast(2 as tinyint) as double) AS (IF(true, CAST(1 AS DOUBLE), CAST(2 AS TINYINT)))#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 double), cast(2 as smallint)) FROM t +-- !query analysis +Project [if (true) cast(1 as double) else cast(cast(2 as smallint) as double) AS (IF(true, CAST(1 AS DOUBLE), CAST(2 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 IF(true, cast(1 as double), cast(2 as int)) FROM t +-- !query analysis +Project [if (true) cast(1 as double) else cast(cast(2 as int) as double) AS (IF(true, CAST(1 AS DOUBLE), CAST(2 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 IF(true, cast(1 as double), cast(2 as bigint)) FROM t +-- !query analysis +Project [if (true) cast(1 as double) else cast(cast(2 as bigint) as double) AS (IF(true, CAST(1 AS DOUBLE), CAST(2 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 IF(true, cast(1 as double), cast(2 as float)) FROM t +-- !query analysis +Project [if (true) cast(1 as double) else cast(cast(2 as float) as double) AS (IF(true, CAST(1 AS DOUBLE), 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 +SELECT IF(true, cast(1 as double), cast(2 as double)) FROM t +-- !query analysis +Project [if (true) cast(1 as double) else cast(2 as double) AS (IF(true, CAST(1 AS DOUBLE), 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 +SELECT IF(true, cast(1 as double), cast(2 as decimal(10, 0))) FROM t +-- !query analysis +Project [if (true) cast(1 as double) else cast(cast(2 as decimal(10,0)) as double) AS (IF(true, CAST(1 AS DOUBLE), CAST(2 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 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] ++- 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 double), cast('2' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DOUBLE), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "IF(true, cast(1 as double), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as double), cast(2 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DOUBLE), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "IF(true, cast(1 as double), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as double), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DOUBLE), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "IF(true, cast(1 as double), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as double), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DOUBLE), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "IF(true, cast(1 as double), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as tinyint)) FROM t +-- !query analysis +Project [if (true) cast(1 as decimal(10,0)) else cast(cast(2 as tinyint) as decimal(10,0)) AS (IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS TINYINT)))#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 decimal(10, 0)), cast(2 as smallint)) FROM t +-- !query analysis +Project [if (true) cast(1 as decimal(10,0)) else cast(cast(2 as smallint) as decimal(10,0)) AS (IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 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 IF(true, cast(1 as decimal(10, 0)), cast(2 as int)) FROM t +-- !query analysis +Project [if (true) cast(1 as decimal(10,0)) else cast(cast(2 as int) as decimal(10,0)) AS (IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 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 IF(true, cast(1 as decimal(10, 0)), cast(2 as bigint)) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as decimal(10,0)) as decimal(20,0)) else cast(cast(2 as bigint) as decimal(20,0)) AS (IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 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 IF(true, cast(1 as decimal(10, 0)), cast(2 as float)) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as decimal(10,0)) as double) else cast(cast(2 as float) as double) AS (IF(true, CAST(1 AS DECIMAL(10,0)), 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 +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as double)) FROM t +-- !query analysis +Project [if (true) cast(cast(1 as decimal(10,0)) as double) else cast(2 as double) AS (IF(true, CAST(1 AS DECIMAL(10,0)), 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 +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as decimal(10, 0))) FROM t +-- !query analysis +Project [if (true) cast(1 as decimal(10,0)) else cast(2 as decimal(10,0)) AS (IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 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 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] ++- 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 decimal(10, 0)), cast('2' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "IF(true, cast(1 as decimal(10, 0)), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "IF(true, cast(1 as decimal(10, 0)), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 86, + "fragment" : "IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 string)) FROM t +-- !query analysis +Project [if (true) cast(1 as string) else cast(2 as string) AS (IF(true, CAST(1 AS STRING), 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 +SELECT IF(true, cast(1 as string), cast('2' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as string), cast(2 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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))" + } ] +} + + +-- !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] ++- 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' 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] ++- 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 binary), cast(2 as tinyint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"TINYINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "IF(true, cast('1' as binary), cast(2 as tinyint))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as smallint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"SMALLINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "IF(true, cast('1' as binary), cast(2 as smallint))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as int)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"INT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "IF(true, cast('1' as binary), cast(2 as int))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as bigint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BIGINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "IF(true, cast('1' as binary), cast(2 as bigint))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as float)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"FLOAT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "IF(true, cast('1' as binary), cast(2 as float))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as double)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DOUBLE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "IF(true, cast('1' as binary), cast(2 as double))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as decimal(10, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DECIMAL(10,0)\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "IF(true, cast('1' as binary), cast(2 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as string)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast('2' as binary)) FROM t +-- !query analysis +Project [if (true) cast(1 as binary) else cast(2 as binary) AS (IF(true, CAST(1 AS BINARY), 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 binary), cast(2 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "IF(true, cast('1' as binary), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "IF(true, cast('1' as binary), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "IF(true, cast('1' as binary), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as tinyint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"TINYINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "IF(true, cast(1 as boolean), cast(2 as tinyint))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as smallint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"SMALLINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "IF(true, cast(1 as boolean), cast(2 as smallint))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as int)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"INT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "IF(true, cast(1 as boolean), cast(2 as int))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as bigint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BIGINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "IF(true, cast(1 as boolean), cast(2 as bigint))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as float)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"FLOAT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "IF(true, cast(1 as boolean), cast(2 as float))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as double)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DOUBLE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "IF(true, cast(1 as boolean), cast(2 as double))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as decimal(10, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DECIMAL(10,0)\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "IF(true, cast(1 as boolean), cast(2 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as string)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast('2' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "IF(true, cast(1 as boolean), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as boolean)) FROM t +-- !query analysis +Project [if (true) cast(1 as boolean) else cast(2 as boolean) AS (IF(true, CAST(1 AS BOOLEAN), 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 boolean), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "IF(true, cast(1 as boolean), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "IF(true, cast(1 as boolean), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as tinyint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TINYINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as tinyint))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as smallint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"SMALLINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as smallint))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as int)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"INT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as int))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as bigint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BIGINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as bigint))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as float)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"FLOAT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as float))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as double)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"DOUBLE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as double))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as decimal(10, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"DECIMAL(10,0)\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 86, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as decimal(10, 0)))" + } ] +} + + +-- !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] ++- 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('2017-12-12 09:30:00.0' as timestamp), cast('2' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +Project [if (true) cast(2017-12-12 09:30:00.0 as timestamp) else cast(2017-12-11 09:30:00.0 as timestamp) AS (IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), 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 +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +Project [if (true) cast(2017-12-12 09:30:00.0 as timestamp) else cast(cast(2017-12-11 09:30:00 as date) as timestamp) AS (IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), 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 +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as tinyint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"TINYINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as tinyint))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as smallint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"SMALLINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as smallint))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as int)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"INT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 68, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as int))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as bigint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BIGINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as bigint))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as float)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"FLOAT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as float))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as double)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DOUBLE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as double))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as decimal(10, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DECIMAL(10,0)\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as decimal(10, 0)))" + } ] +} + + +-- !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] ++- 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('2017-12-12 09:30:00' as date), cast('2' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +Project [if (true) cast(cast(2017-12-12 09:30:00 as date) as timestamp) else cast(2017-12-11 09:30:00.0 as timestamp) AS (IF(true, CAST(2017-12-12 09:30:00 AS DATE), 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 +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +Project [if (true) cast(2017-12-12 09:30:00 as date) else cast(2017-12-11 09:30:00 as date) AS (IF(true, CAST(2017-12-12 09:30:00 AS DATE), 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 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 new file mode 100644 index 0000000000000..9818d54b809a5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/implicitTypeCasts.sql.out @@ -0,0 +1,480 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query analysis +CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT 1 + '2' FROM t +-- !query analysis +Project [(cast(1 as double) + cast(2 as double)) AS (1 + 2)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT 1 - '2' FROM t +-- !query analysis +Project [(cast(1 as double) - cast(2 as double)) AS (1 - 2)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT 1 * '2' FROM t +-- !query analysis +Project [(cast(1 as double) * cast(2 as double)) AS (1 * 2)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT 4 / '2' FROM t +-- !query analysis +Project [(cast(4 as double) / cast(cast(2 as double) as double)) AS (4 / 2)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT 1.1 + '2' FROM t +-- !query analysis +Project [(cast(1.1 as double) + cast(2 as double)) AS (1.1 + 2)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT 1.1 - '2' FROM t +-- !query analysis +Project [(cast(1.1 as double) - cast(2 as double)) AS (1.1 - 2)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT 1.1 * '2' FROM t +-- !query analysis +Project [(cast(1.1 as double) * cast(2 as double)) AS (1.1 * 2)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT 4.4 / '2' FROM t +-- !query analysis +Project [(cast(4.4 as double) / cast(2 as double)) AS (4.4 / 2)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT 1.1 + '2.2' FROM t +-- !query analysis +Project [(cast(1.1 as double) + cast(2.2 as double)) AS (1.1 + 2.2)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT 1.1 - '2.2' FROM t +-- !query analysis +Project [(cast(1.1 as double) - cast(2.2 as double)) AS (1.1 - 2.2)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT 1.1 * '2.2' FROM t +-- !query analysis +Project [(cast(1.1 as double) * cast(2.2 as double)) AS (1.1 * 2.2)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT 4.4 / '2.2' FROM t +-- !query analysis +Project [(cast(4.4 as double) / cast(2.2 as double)) AS (4.4 / 2.2)#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 smallint) || '$' FROM t +-- !query analysis +Project [concat(concat($, cast(cast(1 as smallint) as string)), $) AS concat(concat($, 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 '$' || 1 || '$' FROM t +-- !query analysis +Project [concat(concat($, cast(1 as string)), $) AS concat(concat($, 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 bigint) || '$' FROM t +-- !query analysis +Project [concat(concat($, cast(cast(1 as bigint) as string)), $) AS concat(concat($, 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 '$' || cast(1.1 as float) || '$' FROM t +-- !query analysis +Project [concat(concat($, cast(cast(1.1 as float) as string)), $) AS concat(concat($, CAST(1.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 '$' || cast(1.1 as double) || '$' FROM t +-- !query analysis +Project [concat(concat($, cast(cast(1.1 as double) as string)), $) AS concat(concat($, CAST(1.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 '$' || 1.1 || '$' FROM t +-- !query analysis +Project [concat(concat($, cast(1.1 as string)), $) AS concat(concat($, 1.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.1 as decimal(8,3)) || '$' FROM t +-- !query analysis +Project [concat(concat($, cast(cast(1.1 as decimal(8,3)) as string)), $) AS concat(concat($, CAST(1.1 AS DECIMAL(8,3))), $)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '$' || 'abcd' || '$' FROM t +-- !query analysis +Project [concat(concat($, abcd), $) AS concat(concat($, abcd), $)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '$' || date('1996-09-09') || '$' FROM t +-- !query analysis +Project [concat(concat($, cast(cast(1996-09-09 as date) as string)), $) AS concat(concat($, 1996-09-09), $)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '$' || timestamp('1996-09-09 10:11:12.4' )|| '$' FROM t +-- !query analysis +Project [concat(concat($, cast(cast(1996-09-09 10:11:12.4 as timestamp) as string)), $) AS concat(concat($, 1996-09-09 10:11:12.4), $)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT length(cast(1 as smallint)) FROM t +-- !query analysis +Project [length(cast(cast(1 as smallint) as string)) AS length(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 length(cast(1 as int)) FROM t +-- !query analysis +Project [length(cast(cast(1 as int) as string)) AS length(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 length(cast(1 as bigint)) FROM t +-- !query analysis +Project [length(cast(cast(1 as bigint) as string)) AS length(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 length(cast(1.1 as float)) FROM t +-- !query analysis +Project [length(cast(cast(1.1 as float) as string)) AS length(CAST(1.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 length(cast(1.1 as double)) FROM t +-- !query analysis +Project [length(cast(cast(1.1 as double) as string)) AS length(CAST(1.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 length(1.1) FROM t +-- !query analysis +Project [length(cast(1.1 as string)) AS length(1.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 length(cast(1.1 as decimal(8,3))) FROM t +-- !query analysis +Project [length(cast(cast(1.1 as decimal(8,3)) as string)) AS length(CAST(1.1 AS DECIMAL(8,3)))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT length('four') FROM t +-- !query analysis +Project [length(four) AS length(four)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT length(date('1996-09-10')) FROM t +-- !query analysis +Project [length(cast(cast(1996-09-10 as date) as string)) AS length(1996-09-10)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT length(timestamp('1996-09-10 10:11:12.4')) FROM t +-- !query analysis +Project [length(cast(cast(1996-09-10 10:11:12.4 as timestamp) as string)) AS length(1996-09-10 10:11:12.4)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT year( '1996-01-10') FROM t +-- !query analysis +Project [year(cast(1996-01-10 as date)) AS year(1996-01-10)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT month( '1996-01-10') FROM t +-- !query analysis +Project [month(cast(1996-01-10 as date)) AS month(1996-01-10)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT day( '1996-01-10') FROM t +-- !query analysis +Project [day(cast(1996-01-10 as date)) AS day(1996-01-10)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT hour( '10:11:12') FROM t +-- !query analysis +Project [hour(cast(10:11:12 as timestamp), Some(America/Los_Angeles)) AS hour(10:11:12)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT minute( '10:11:12') FROM t +-- !query analysis +Project [minute(cast(10:11:12 as timestamp), Some(America/Los_Angeles)) AS minute(10:11:12)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT second( '10:11:12') FROM t +-- !query analysis +Project [second(cast(10:11:12 as timestamp), Some(America/Los_Angeles)) AS second(10:11:12)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +select 1 like '%' FROM t +-- !query analysis +Project [cast(1 as string) LIKE % AS 1 LIKE %#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +select date('1996-09-10') like '19%' FROM t +-- !query analysis +Project [cast(cast(1996-09-10 as date) as string) LIKE 19% AS 1996-09-10 LIKE 19%#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +select '1' like 1 FROM t +-- !query analysis +Project [1 LIKE cast(1 as string) AS 1 LIKE 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 '1 ' like 1 FROM t +-- !query analysis +Project [1 LIKE cast(1 as string) AS 1 LIKE 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 '1996-09-10' like date('1996-09-10') FROM t +-- !query analysis +Project [1996-09-10 LIKE cast(cast(1996-09-10 as date) as string) AS 1996-09-10 LIKE 1996-09-10#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation 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 new file mode 100644 index 0000000000000..ae8c3f6479331 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/inConversion.sql.out @@ -0,0 +1,4715 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query analysis +CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint)) FROM t +-- !query analysis +Project [cast(1 as tinyint) IN (cast(1 as tinyint)) AS (CAST(1 AS TINYINT) IN (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 + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as smallint)) FROM t +-- !query analysis +Project [cast(cast(1 as tinyint) as smallint) IN (cast(cast(1 as smallint) as smallint)) AS (CAST(1 AS TINYINT) IN (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 cast(1 as tinyint) in (cast(1 as int)) FROM t +-- !query analysis +Project [cast(cast(1 as tinyint) as int) IN (cast(cast(1 as int) as int)) AS (CAST(1 AS TINYINT) IN (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 cast(1 as tinyint) in (cast(1 as bigint)) FROM t +-- !query analysis +Project [cast(cast(1 as tinyint) as bigint) IN (cast(cast(1 as bigint) as bigint)) AS (CAST(1 AS TINYINT) IN (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 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] ++- 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) in (cast(1 as double)) FROM t +-- !query analysis +Project [cast(cast(1 as tinyint) as double) IN (cast(cast(1 as double) as double)) AS (CAST(1 AS TINYINT) IN (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 cast(1 as tinyint) in (cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [cast(cast(1 as tinyint) as decimal(10,0)) IN (cast(cast(1 as decimal(10,0)) as decimal(10,0))) AS (CAST(1 AS TINYINT) IN (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 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] ++- 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) in (cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 50, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 49, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 73, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 66, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as tinyint)) FROM t +-- !query analysis +Project [cast(cast(1 as smallint) as smallint) IN (cast(cast(1 as tinyint) as smallint)) AS (CAST(1 AS SMALLINT) IN (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 + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint)) FROM t +-- !query analysis +Project [cast(1 as smallint) IN (cast(1 as smallint)) AS (CAST(1 AS SMALLINT) IN (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 cast(1 as smallint) in (cast(1 as int)) FROM t +-- !query analysis +Project [cast(cast(1 as smallint) as int) IN (cast(cast(1 as int) as int)) AS (CAST(1 AS SMALLINT) IN (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 cast(1 as smallint) in (cast(1 as bigint)) FROM t +-- !query analysis +Project [cast(cast(1 as smallint) as bigint) IN (cast(cast(1 as bigint) as bigint)) AS (CAST(1 AS SMALLINT) IN (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 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] ++- 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 smallint) in (cast(1 as double)) FROM t +-- !query analysis +Project [cast(cast(1 as smallint) as double) IN (cast(cast(1 as double) as double)) AS (CAST(1 AS SMALLINT) IN (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 cast(1 as smallint) in (cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [cast(cast(1 as smallint) as decimal(10,0)) IN (cast(cast(1 as decimal(10,0)) as decimal(10,0))) AS (CAST(1 AS SMALLINT) IN (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 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] ++- 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 smallint) in (cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 51, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 50, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 74, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 67, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast(1 as tinyint)) FROM t +-- !query analysis +Project [cast(cast(1 as int) as int) IN (cast(cast(1 as tinyint) as int)) AS (CAST(1 AS INT) IN (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 + + +-- !query +SELECT cast(1 as int) in (cast(1 as smallint)) FROM t +-- !query analysis +Project [cast(cast(1 as int) as int) IN (cast(cast(1 as smallint) as int)) AS (CAST(1 AS INT) IN (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 cast(1 as int) in (cast(1 as int)) FROM t +-- !query analysis +Project [cast(1 as int) IN (cast(1 as int)) AS (CAST(1 AS INT) IN (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 cast(1 as int) in (cast(1 as bigint)) FROM t +-- !query analysis +Project [cast(cast(1 as int) as bigint) IN (cast(cast(1 as bigint) as bigint)) AS (CAST(1 AS INT) IN (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 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] ++- 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 int) in (cast(1 as double)) FROM t +-- !query analysis +Project [cast(cast(1 as int) as double) IN (cast(cast(1 as double) as double)) AS (CAST(1 AS INT) IN (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 cast(1 as int) in (cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [cast(cast(1 as int) as decimal(10,0)) IN (cast(cast(1 as decimal(10,0)) as decimal(10,0))) AS (CAST(1 AS INT) IN (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 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] ++- 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 int) in (cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 46, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 45, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 69, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 62, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as tinyint)) FROM t +-- !query analysis +Project [cast(cast(1 as bigint) as bigint) IN (cast(cast(1 as tinyint) as bigint)) AS (CAST(1 AS BIGINT) IN (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 + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as smallint)) FROM t +-- !query analysis +Project [cast(cast(1 as bigint) as bigint) IN (cast(cast(1 as smallint) as bigint)) AS (CAST(1 AS BIGINT) IN (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 cast(1 as bigint) in (cast(1 as int)) FROM t +-- !query analysis +Project [cast(cast(1 as bigint) as bigint) IN (cast(cast(1 as int) as bigint)) AS (CAST(1 AS BIGINT) IN (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 cast(1 as bigint) in (cast(1 as bigint)) FROM t +-- !query analysis +Project [cast(1 as bigint) IN (cast(1 as bigint)) AS (CAST(1 AS BIGINT) IN (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 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] ++- 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 bigint) in (cast(1 as double)) FROM t +-- !query analysis +Project [cast(cast(1 as bigint) as double) IN (cast(cast(1 as double) as double)) AS (CAST(1 AS BIGINT) IN (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 cast(1 as bigint) in (cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [cast(cast(1 as bigint) as decimal(20,0)) IN (cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) IN (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 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] ++- 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 bigint) in (cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 49, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 48, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 72, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 65, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 float) in (cast(1 as float)) FROM t +-- !query analysis +Project [cast(1 as float) IN (cast(1 as float)) AS (CAST(1 AS FLOAT) IN (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 cast(1 as float) in (cast(1 as double)) FROM t +-- !query analysis +Project [cast(cast(1 as float) as double) IN (cast(cast(1 as double) as double)) AS (CAST(1 AS FLOAT) IN (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 cast(1 as float) in (cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [cast(cast(1 as float) as double) IN (cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS FLOAT) IN (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 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] ++- 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 float) in (cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 48, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as float) in (cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 47, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as float) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 71, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as float) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 64, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast(1 as tinyint)) FROM t +-- !query analysis +Project [cast(cast(1 as double) as double) IN (cast(cast(1 as tinyint) as double)) AS (CAST(1 AS DOUBLE) IN (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 + + +-- !query +SELECT cast(1 as double) in (cast(1 as smallint)) FROM t +-- !query analysis +Project [cast(cast(1 as double) as double) IN (cast(cast(1 as smallint) as double)) AS (CAST(1 AS DOUBLE) IN (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 cast(1 as double) in (cast(1 as int)) FROM t +-- !query analysis +Project [cast(cast(1 as double) as double) IN (cast(cast(1 as int) as double)) AS (CAST(1 AS DOUBLE) IN (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 cast(1 as double) in (cast(1 as bigint)) FROM t +-- !query analysis +Project [cast(cast(1 as double) as double) IN (cast(cast(1 as bigint) as double)) AS (CAST(1 AS DOUBLE) IN (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 cast(1 as double) in (cast(1 as float)) FROM t +-- !query analysis +Project [cast(cast(1 as double) as double) IN (cast(cast(1 as float) as double)) AS (CAST(1 AS DOUBLE) IN (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 cast(1 as double) in (cast(1 as double)) FROM t +-- !query analysis +Project [cast(1 as double) IN (cast(1 as double)) AS (CAST(1 AS DOUBLE) IN (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 cast(1 as double) in (cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [cast(cast(1 as double) as double) IN (cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS DOUBLE) IN (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 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] ++- 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 double) in (cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 49, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 48, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 72, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 65, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as tinyint)) FROM t +-- !query analysis +Project [cast(cast(1 as decimal(10,0)) as decimal(10,0)) IN (cast(cast(1 as tinyint) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) IN (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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as smallint)) FROM t +-- !query analysis +Project [cast(cast(1 as decimal(10,0)) as decimal(10,0)) IN (cast(cast(1 as smallint) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) IN (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 cast(1 as decimal(10, 0)) in (cast(1 as int)) FROM t +-- !query analysis +Project [cast(cast(1 as decimal(10,0)) as decimal(10,0)) IN (cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) IN (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 cast(1 as decimal(10, 0)) in (cast(1 as bigint)) FROM t +-- !query analysis +Project [cast(cast(1 as decimal(10,0)) as decimal(20,0)) IN (cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) IN (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 cast(1 as decimal(10, 0)) in (cast(1 as float)) FROM t +-- !query analysis +Project [cast(cast(1 as decimal(10,0)) as double) IN (cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(10,0)) IN (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 cast(1 as decimal(10, 0)) in (cast(1 as double)) FROM t +-- !query analysis +Project [cast(cast(1 as decimal(10,0)) as double) IN (cast(cast(1 as double) as double)) AS (CAST(1 AS DECIMAL(10,0)) IN (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 cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [cast(1 as decimal(10,0)) IN (cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) IN (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 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] ++- 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 decimal(10, 0)) in (cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 57, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 56, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 80, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 73, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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)) FROM t +-- !query analysis +Project [cast(1 as string) IN (cast(1 as string)) AS (CAST(1 AS STRING) 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 +SELECT cast(1 as string) in (cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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))" + } ] +} + + +-- !query +SELECT cast(1 as string) in (cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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))" + } ] +} + + +-- !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] ++- 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' 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] ++- 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 binary) in (cast(1 as tinyint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 50, + "fragment" : "in (cast(1 as tinyint))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast(1 as smallint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 51, + "fragment" : "in (cast(1 as smallint))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast(1 as int)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 46, + "fragment" : "in (cast(1 as int))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast(1 as bigint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 49, + "fragment" : "in (cast(1 as bigint))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast(1 as float)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 48, + "fragment" : "in (cast(1 as float))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast(1 as double)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 49, + "fragment" : "in (cast(1 as double))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast(1 as decimal(10, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 57, + "fragment" : "in (cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast(1 as string)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary)) FROM t +-- !query analysis +Project [cast(1 as binary) IN (cast(1 as binary)) AS (CAST(1 AS BINARY) 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 binary) in (cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 50, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 74, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 67, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT true in (cast(1 as tinyint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 35, + "fragment" : "in (cast(1 as tinyint))" + } ] +} + + +-- !query +SELECT true in (cast(1 as smallint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 36, + "fragment" : "in (cast(1 as smallint))" + } ] +} + + +-- !query +SELECT true in (cast(1 as int)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 31, + "fragment" : "in (cast(1 as int))" + } ] +} + + +-- !query +SELECT true in (cast(1 as bigint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 34, + "fragment" : "in (cast(1 as bigint))" + } ] +} + + +-- !query +SELECT true in (cast(1 as float)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 33, + "fragment" : "in (cast(1 as float))" + } ] +} + + +-- !query +SELECT true in (cast(1 as double)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 34, + "fragment" : "in (cast(1 as double))" + } ] +} + + +-- !query +SELECT true in (cast(1 as decimal(10, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 42, + "fragment" : "in (cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT true in (cast(1 as string)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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))" + } ] +} + + +-- !query +SELECT true in (cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 36, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT true in (cast(1 as boolean)) FROM t +-- !query analysis +Project [true IN (cast(1 as boolean)) AS (true 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 true in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 59, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT true in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 52, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as tinyint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 73, + "fragment" : "in (cast(2 as tinyint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as smallint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 74, + "fragment" : "in (cast(2 as smallint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as int)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 69, + "fragment" : "in (cast(2 as int))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as bigint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 72, + "fragment" : "in (cast(2 as bigint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as float)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 71, + "fragment" : "in (cast(2 as float))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as double)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 72, + "fragment" : "in (cast(2 as double))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as decimal(10, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 80, + "fragment" : "in (cast(2 as decimal(10, 0)))" + } ] +} + + +-- !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] ++- 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-12 09:30:00.0' as timestamp) in (cast('2' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 74, + "fragment" : "in (cast('2' as binary))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 73, + "fragment" : "in (cast(2 as boolean))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +Project [cast(2017-12-12 09:30:00.0 as timestamp) IN (cast(2017-12-11 09:30:00.0 as timestamp)) AS (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) 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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +Project [cast(cast(2017-12-12 09:30:00.0 as timestamp) as timestamp) IN (cast(cast(2017-12-11 09:30:00 as date) as timestamp)) AS (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) 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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as tinyint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 66, + "fragment" : "in (cast(2 as tinyint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as smallint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 67, + "fragment" : "in (cast(2 as smallint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as int)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 62, + "fragment" : "in (cast(2 as int))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as bigint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 65, + "fragment" : "in (cast(2 as bigint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as float)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 64, + "fragment" : "in (cast(2 as float))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as double)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 65, + "fragment" : "in (cast(2 as double))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as decimal(10, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 73, + "fragment" : "in (cast(2 as decimal(10, 0)))" + } ] +} + + +-- !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] ++- 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-12 09:30:00' as date) in (cast('2' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 67, + "fragment" : "in (cast('2' as binary))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 66, + "fragment" : "in (cast(2 as boolean))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +Project [cast(cast(2017-12-12 09:30:00 as date) as timestamp) IN (cast(cast(2017-12-11 09:30:00.0 as timestamp) as timestamp)) AS (CAST(2017-12-12 09:30:00 AS DATE) 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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +Project [cast(2017-12-12 09:30:00 as date) IN (cast(2017-12-11 09:30:00 as date)) AS (CAST(2017-12-12 09:30:00 AS DATE) 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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as tinyint)) FROM t +-- !query analysis +Project [cast(1 as tinyint) IN (cast(1 as tinyint),cast(1 as tinyint)) AS (CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), 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 + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as smallint)) FROM t +-- !query analysis +Project [cast(cast(1 as tinyint) as smallint) IN (cast(cast(1 as tinyint) as smallint),cast(cast(1 as smallint) as smallint)) AS (CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), 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 cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as int)) FROM t +-- !query analysis +Project [cast(cast(1 as tinyint) as int) IN (cast(cast(1 as tinyint) as int),cast(cast(1 as int) as int)) AS (CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), 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 cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as bigint)) FROM t +-- !query analysis +Project [cast(cast(1 as tinyint) as bigint) IN (cast(cast(1 as tinyint) as bigint),cast(cast(1 as bigint) as bigint)) AS (CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), 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 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] ++- 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) in (cast(1 as tinyint), cast(1 as double)) FROM t +-- !query analysis +Project [cast(cast(1 as tinyint) as double) IN (cast(cast(1 as tinyint) as double),cast(cast(1 as double) as double)) AS (CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), 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 cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [cast(cast(1 as tinyint) as decimal(10,0)) IN (cast(cast(1 as tinyint) as decimal(10,0)),cast(cast(1 as decimal(10,0)) as decimal(10,0))) AS (CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), 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 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] ++- 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) in (cast(1 as tinyint), cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"TINYINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 70, + "fragment" : "in (cast(1 as tinyint), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"TINYINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 69, + "fragment" : "in (cast(1 as tinyint), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"TINYINT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 93, + "fragment" : "in (cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"TINYINT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 86, + "fragment" : "in (cast(1 as tinyint), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as tinyint)) FROM t +-- !query analysis +Project [cast(cast(1 as smallint) as smallint) IN (cast(cast(1 as smallint) as smallint),cast(cast(1 as tinyint) as smallint)) AS (CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), 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 + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as smallint)) FROM t +-- !query analysis +Project [cast(1 as smallint) IN (cast(1 as smallint),cast(1 as smallint)) AS (CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), 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 cast(1 as smallint) in (cast(1 as smallint), cast(1 as int)) FROM t +-- !query analysis +Project [cast(cast(1 as smallint) as int) IN (cast(cast(1 as smallint) as int),cast(cast(1 as int) as int)) AS (CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), 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 cast(1 as smallint) in (cast(1 as smallint), cast(1 as bigint)) FROM t +-- !query analysis +Project [cast(cast(1 as smallint) as bigint) IN (cast(cast(1 as smallint) as bigint),cast(cast(1 as bigint) as bigint)) AS (CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), 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 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] ++- 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 smallint) in (cast(1 as smallint), cast(1 as double)) FROM t +-- !query analysis +Project [cast(cast(1 as smallint) as double) IN (cast(cast(1 as smallint) as double),cast(cast(1 as double) as double)) AS (CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), 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 cast(1 as smallint) in (cast(1 as smallint), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [cast(cast(1 as smallint) as decimal(10,0)) IN (cast(cast(1 as smallint) as decimal(10,0)),cast(cast(1 as decimal(10,0)) as decimal(10,0))) AS (CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), 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 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] ++- 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 smallint) in (cast(1 as smallint), cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"SMALLINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 72, + "fragment" : "in (cast(1 as smallint), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"SMALLINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 71, + "fragment" : "in (cast(1 as smallint), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"SMALLINT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 95, + "fragment" : "in (cast(1 as smallint), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"SMALLINT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 88, + "fragment" : "in (cast(1 as smallint), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast(1 as tinyint)) FROM t +-- !query analysis +Project [cast(cast(1 as int) as int) IN (cast(cast(1 as int) as int),cast(cast(1 as tinyint) as int)) AS (CAST(1 AS INT) IN (CAST(1 AS INT), 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 + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast(1 as smallint)) FROM t +-- !query analysis +Project [cast(cast(1 as int) as int) IN (cast(cast(1 as int) as int),cast(cast(1 as smallint) as int)) AS (CAST(1 AS INT) IN (CAST(1 AS INT), 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 cast(1 as int) in (cast(1 as int), cast(1 as int)) FROM t +-- !query analysis +Project [cast(1 as int) IN (cast(1 as int),cast(1 as int)) AS (CAST(1 AS INT) IN (CAST(1 AS INT), 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 cast(1 as int) in (cast(1 as int), cast(1 as bigint)) FROM t +-- !query analysis +Project [cast(cast(1 as int) as bigint) IN (cast(cast(1 as int) as bigint),cast(cast(1 as bigint) as bigint)) AS (CAST(1 AS INT) IN (CAST(1 AS INT), 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 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] ++- 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 int) in (cast(1 as int), cast(1 as double)) FROM t +-- !query analysis +Project [cast(cast(1 as int) as double) IN (cast(cast(1 as int) as double),cast(cast(1 as double) as double)) AS (CAST(1 AS INT) IN (CAST(1 AS INT), 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 cast(1 as int) in (cast(1 as int), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [cast(cast(1 as int) as decimal(10,0)) IN (cast(cast(1 as int) as decimal(10,0)),cast(cast(1 as decimal(10,0)) as decimal(10,0))) AS (CAST(1 AS INT) IN (CAST(1 AS INT), 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 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] ++- 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 int) in (cast(1 as int), cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"INT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 62, + "fragment" : "in (cast(1 as int), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"INT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 61, + "fragment" : "in (cast(1 as int), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"INT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 85, + "fragment" : "in (cast(1 as int), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"INT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 78, + "fragment" : "in (cast(1 as int), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as tinyint)) FROM t +-- !query analysis +Project [cast(cast(1 as bigint) as bigint) IN (cast(cast(1 as bigint) as bigint),cast(cast(1 as tinyint) as bigint)) AS (CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), 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 + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as smallint)) FROM t +-- !query analysis +Project [cast(cast(1 as bigint) as bigint) IN (cast(cast(1 as bigint) as bigint),cast(cast(1 as smallint) as bigint)) AS (CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), 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 cast(1 as bigint) in (cast(1 as bigint), cast(1 as int)) FROM t +-- !query analysis +Project [cast(cast(1 as bigint) as bigint) IN (cast(cast(1 as bigint) as bigint),cast(cast(1 as int) as bigint)) AS (CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), 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 cast(1 as bigint) in (cast(1 as bigint), cast(1 as bigint)) FROM t +-- !query analysis +Project [cast(1 as bigint) IN (cast(1 as bigint),cast(1 as bigint)) AS (CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), 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 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] ++- 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 bigint) in (cast(1 as bigint), cast(1 as double)) FROM t +-- !query analysis +Project [cast(cast(1 as bigint) as double) IN (cast(cast(1 as bigint) as double),cast(cast(1 as double) as double)) AS (CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), 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 cast(1 as bigint) in (cast(1 as bigint), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [cast(cast(1 as bigint) as decimal(20,0)) IN (cast(cast(1 as bigint) as decimal(20,0)),cast(cast(1 as decimal(10,0)) as decimal(20,0))) AS (CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), 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 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] ++- 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 bigint) in (cast(1 as bigint), cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BIGINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 68, + "fragment" : "in (cast(1 as bigint), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BIGINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 67, + "fragment" : "in (cast(1 as bigint), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BIGINT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 91, + "fragment" : "in (cast(1 as bigint), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BIGINT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 84, + "fragment" : "in (cast(1 as bigint), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 float) in (cast(1 as float), cast(1 as float)) FROM t +-- !query analysis +Project [cast(1 as float) IN (cast(1 as float),cast(1 as float)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), 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 cast(1 as float) in (cast(1 as float), cast(1 as double)) FROM t +-- !query analysis +Project [cast(cast(1 as float) as double) IN (cast(cast(1 as float) as double),cast(cast(1 as double) as double)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), 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 cast(1 as float) in (cast(1 as float), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [cast(cast(1 as float) as double) IN (cast(cast(1 as float) as double),cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), 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 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] ++- 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 float) in (cast(1 as float), cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"FLOAT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 66, + "fragment" : "in (cast(1 as float), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as float) in (cast(1 as float), cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"FLOAT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 65, + "fragment" : "in (cast(1 as float), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as float) in (cast(1 as float), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"FLOAT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 89, + "fragment" : "in (cast(1 as float), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as float) in (cast(1 as float), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"FLOAT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 82, + "fragment" : "in (cast(1 as float), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast(1 as tinyint)) FROM t +-- !query analysis +Project [cast(cast(1 as double) as double) IN (cast(cast(1 as double) as double),cast(cast(1 as tinyint) as double)) AS (CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), 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 + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast(1 as smallint)) FROM t +-- !query analysis +Project [cast(cast(1 as double) as double) IN (cast(cast(1 as double) as double),cast(cast(1 as smallint) as double)) AS (CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), 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 cast(1 as double) in (cast(1 as double), cast(1 as int)) FROM t +-- !query analysis +Project [cast(cast(1 as double) as double) IN (cast(cast(1 as double) as double),cast(cast(1 as int) as double)) AS (CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), 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 cast(1 as double) in (cast(1 as double), cast(1 as bigint)) FROM t +-- !query analysis +Project [cast(cast(1 as double) as double) IN (cast(cast(1 as double) as double),cast(cast(1 as bigint) as double)) AS (CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), 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 cast(1 as double) in (cast(1 as double), cast(1 as float)) FROM t +-- !query analysis +Project [cast(cast(1 as double) as double) IN (cast(cast(1 as double) as double),cast(cast(1 as float) as double)) AS (CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), 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 cast(1 as double) in (cast(1 as double), cast(1 as double)) FROM t +-- !query analysis +Project [cast(1 as double) IN (cast(1 as double),cast(1 as double)) AS (CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), 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 cast(1 as double) in (cast(1 as double), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [cast(cast(1 as double) as double) IN (cast(cast(1 as double) as double),cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), 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 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] ++- 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 double) in (cast(1 as double), cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"DOUBLE\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 68, + "fragment" : "in (cast(1 as double), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"DOUBLE\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 67, + "fragment" : "in (cast(1 as double), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"DOUBLE\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 91, + "fragment" : "in (cast(1 as double), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"DOUBLE\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 84, + "fragment" : "in (cast(1 as double), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as tinyint)) FROM t +-- !query analysis +Project [cast(cast(1 as decimal(10,0)) as decimal(10,0)) IN (cast(cast(1 as decimal(10,0)) as decimal(10,0)),cast(cast(1 as tinyint) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), 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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as smallint)) FROM t +-- !query analysis +Project [cast(cast(1 as decimal(10,0)) as decimal(10,0)) IN (cast(cast(1 as decimal(10,0)) as decimal(10,0)),cast(cast(1 as smallint) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), 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 cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as int)) FROM t +-- !query analysis +Project [cast(cast(1 as decimal(10,0)) as decimal(10,0)) IN (cast(cast(1 as decimal(10,0)) as decimal(10,0)),cast(cast(1 as int) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), 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 cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as bigint)) FROM t +-- !query analysis +Project [cast(cast(1 as decimal(10,0)) as decimal(20,0)) IN (cast(cast(1 as decimal(10,0)) as decimal(20,0)),cast(cast(1 as bigint) as decimal(20,0))) AS (CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), 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 cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as float)) FROM t +-- !query analysis +Project [cast(cast(1 as decimal(10,0)) as double) IN (cast(cast(1 as decimal(10,0)) as double),cast(cast(1 as float) as double)) AS (CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), 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 cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as double)) FROM t +-- !query analysis +Project [cast(cast(1 as decimal(10,0)) as double) IN (cast(cast(1 as decimal(10,0)) as double),cast(cast(1 as double) as double)) AS (CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), 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 cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [cast(1 as decimal(10,0)) IN (cast(1 as decimal(10,0)),cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), 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 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] ++- 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 decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"DECIMAL(10,0)\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 84, + "fragment" : "in (cast(1 as decimal(10, 0)), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"DECIMAL(10,0)\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 83, + "fragment" : "in (cast(1 as decimal(10, 0)), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"DECIMAL(10,0)\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 107, + "fragment" : "in (cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"DECIMAL(10,0)\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 100, + "fragment" : "in (cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 string)) FROM t +-- !query analysis +Project [cast(1 as string) IN (cast(1 as string),cast(1 as string)) AS (CAST(1 AS STRING) IN (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 + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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))" + } ] +} + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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))" + } ] +} + + +-- !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] ++- 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' 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] ++- 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 binary) in (cast('1' as binary), cast(1 as tinyint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 71, + "fragment" : "in (cast('1' as binary), cast(1 as tinyint))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as smallint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 72, + "fragment" : "in (cast('1' as binary), cast(1 as smallint))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as int)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 67, + "fragment" : "in (cast('1' as binary), cast(1 as int))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as bigint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 70, + "fragment" : "in (cast('1' as binary), cast(1 as bigint))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as float)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 69, + "fragment" : "in (cast('1' as binary), cast(1 as float))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as double)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 70, + "fragment" : "in (cast('1' as binary), cast(1 as double))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 78, + "fragment" : "in (cast('1' as binary), cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as string)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast('1' as binary)) FROM t +-- !query analysis +Project [cast(1 as binary) IN (cast(1 as binary),cast(1 as binary)) AS (CAST(1 AS BINARY) IN (CAST(1 AS BINARY), 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 binary) in (cast('1' as binary), cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 71, + "fragment" : "in (cast('1' as binary), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 95, + "fragment" : "in (cast('1' as binary), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 88, + "fragment" : "in (cast('1' as binary), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as tinyint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 73, + "fragment" : "in (cast('1' as boolean), cast(1 as tinyint))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as smallint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 74, + "fragment" : "in (cast('1' as boolean), cast(1 as smallint))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as int)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 69, + "fragment" : "in (cast('1' as boolean), cast(1 as int))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as bigint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 72, + "fragment" : "in (cast('1' as boolean), cast(1 as bigint))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as float)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 71, + "fragment" : "in (cast('1' as boolean), cast(1 as float))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as double)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 72, + "fragment" : "in (cast('1' as boolean), cast(1 as double))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 80, + "fragment" : "in (cast('1' as boolean), cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as string)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 74, + "fragment" : "in (cast('1' as boolean), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as boolean)) FROM t +-- !query analysis +Project [cast(1 as boolean) IN (cast(1 as boolean),cast(1 as boolean)) AS (CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), 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 boolean) in (cast('1' as boolean), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 97, + "fragment" : "in (cast('1' as boolean), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 90, + "fragment" : "in (cast('1' as boolean), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !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 tinyint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 117, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as tinyint))" + } ] +} + + +-- !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 smallint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 118, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as smallint))" + } ] +} + + +-- !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 int)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 113, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as int))" + } ] +} + + +-- !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 bigint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 116, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as bigint))" + } ] +} + + +-- !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 float)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 115, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as float))" + } ] +} + + +-- !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 double)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 116, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as double))" + } ] +} + + +-- !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 decimal(10, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 124, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as decimal(10, 0)))" + } ] +} + + +-- !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] ++- 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-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 118, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast('1' as binary))" + } ] +} + + +-- !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 boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 117, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +Project [cast(2017-12-12 09:30:00.0 as timestamp) IN (cast(2017-12-12 09:30:00.0 as timestamp),cast(2017-12-11 09:30:00.0 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(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 +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +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(2017-12-11 09:30:00 as date) 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(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 +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as tinyint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 103, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as tinyint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as smallint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 104, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as smallint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as int)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 99, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as int))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as bigint)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 102, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as bigint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as float)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 101, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as float))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as double)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 102, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as double))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as decimal(10, 0))) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 110, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as decimal(10, 0)))" + } ] +} + + +-- !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] ++- 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-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 104, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 103, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +Project [cast(cast(2017-12-12 09:30:00 as date) as timestamp) IN (cast(cast(2017-12-12 09:30:00 as date) as timestamp),cast(cast(2017-12-11 09:30:00.0 as timestamp) as timestamp)) AS (CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), 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 +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +Project [cast(2017-12-12 09:30:00 as date) IN (cast(2017-12-12 09:30:00 as date),cast(2017-12-11 09:30:00 as date)) AS (CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), 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 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 new file mode 100644 index 0000000000000..0ab21df6b65fd --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapZipWith.sql.out @@ -0,0 +1,289 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( + map(true, false), + map(2Y, 1Y), + map(2S, 1S), + map(2, 1), + map(2L, 1L), + map(922337203685477897945456575809789456, 922337203685477897945456575809789456), + map(9.22337203685477897945456575809789456, 9.22337203685477897945456575809789456), + map(2.0D, 1.0D), + map(float(2.0), float(1.0)), + map(date '2016-03-14', date '2016-03-13'), + map(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), + map('true', 'false', '2', '1'), + map('2016-03-14', '2016-03-13'), + map('2016-11-15 20:54:00.000', '2016-11-12 20:54:00.000'), + map('922337203685477897945456575809789456', 'text'), + map(array(1L, 2L), array(1L, 2L)), map(array(1, 2), array(1, 2)), + map(struct(1S, 2L), struct(1S, 2L)), map(struct(1, 2), struct(1, 2)) +) AS various_maps( + boolean_map, + tinyint_map, + smallint_map, + int_map, + bigint_map, + decimal_map1, decimal_map2, + double_map, + float_map, + date_map, + timestamp_map, + string_map1, string_map2, string_map3, string_map4, + array_map1, array_map2, + struct_map1, struct_map2 +) +-- !query analysis +CreateViewCommand `various_maps`, SELECT * FROM VALUES ( + map(true, false), + map(2Y, 1Y), + map(2S, 1S), + map(2, 1), + map(2L, 1L), + map(922337203685477897945456575809789456, 922337203685477897945456575809789456), + map(9.22337203685477897945456575809789456, 9.22337203685477897945456575809789456), + map(2.0D, 1.0D), + map(float(2.0), float(1.0)), + map(date '2016-03-14', date '2016-03-13'), + map(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), + map('true', 'false', '2', '1'), + map('2016-03-14', '2016-03-13'), + map('2016-11-15 20:54:00.000', '2016-11-12 20:54:00.000'), + map('922337203685477897945456575809789456', 'text'), + map(array(1L, 2L), array(1L, 2L)), map(array(1, 2), array(1, 2)), + map(struct(1S, 2L), struct(1S, 2L)), map(struct(1, 2), struct(1, 2)) +) AS various_maps( + boolean_map, + tinyint_map, + smallint_map, + int_map, + bigint_map, + decimal_map1, decimal_map2, + double_map, + float_map, + date_map, + timestamp_map, + string_map1, string_map2, string_map3, string_map4, + array_map1, array_map2, + struct_map1, struct_map2 +), false, false, LocalTempView, true + +- 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] + + +-- !query +SELECT map_zip_with(tinyint_map, smallint_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query analysis +Project [map_zip_with(cast(tinyint_map#x as map), smallint_map#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] + + +-- !query +SELECT map_zip_with(smallint_map, int_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query analysis +Project [map_zip_with(cast(smallint_map#x as map), int_map#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] + + +-- !query +SELECT map_zip_with(int_map, bigint_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query analysis +Project [map_zip_with(cast(int_map#x as map), bigint_map#x, lambdafunction(struct(k, lambda k#xL, v1, lambda v1#x, v2, lambda v2#xL), lambda k#xL, lambda v1#x, lambda v2#xL, 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] + + +-- !query +SELECT map_zip_with(double_map, float_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query analysis +Project [map_zip_with(double_map#x, cast(float_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] + + +-- !query +SELECT map_zip_with(decimal_map1, decimal_map2, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`map_zip_with`", + "leftType" : "\"DECIMAL(36,0)\"", + "rightType" : "\"DECIMAL(36,35)\"", + "sqlExpr" : "\"map_zip_with(decimal_map1, decimal_map2, lambdafunction(struct(k, v1, v2), k, v1, v2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 81, + "fragment" : "map_zip_with(decimal_map1, decimal_map2, (k, v1, v2) -> struct(k, v1, v2))" + } ] +} + + +-- !query +SELECT map_zip_with(decimal_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query analysis +Project [map_zip_with(decimal_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] + + +-- !query +SELECT map_zip_with(decimal_map1, double_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query analysis +Project [map_zip_with(cast(decimal_map1#x as map), double_map#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] + + +-- !query +SELECT map_zip_with(decimal_map2, int_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`map_zip_with`", + "leftType" : "\"DECIMAL(36,35)\"", + "rightType" : "\"INT\"", + "sqlExpr" : "\"map_zip_with(decimal_map2, int_map, lambdafunction(struct(k, v1, v2), k, v1, v2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "map_zip_with(decimal_map2, int_map, (k, v1, v2) -> struct(k, v1, v2))" + } ] +} + + +-- !query +SELECT map_zip_with(decimal_map2, double_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query analysis +Project [map_zip_with(cast(decimal_map2#x as map), double_map#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] + + +-- !query +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] + + +-- !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] + + +-- !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] + + +-- !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] + + +-- !query +SELECT map_zip_with(array_map1, array_map2, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query analysis +Project [map_zip_with(array_map1#x, cast(array_map2#x as map,array>), 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] + + +-- !query +SELECT map_zip_with(struct_map1, struct_map2, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query analysis +Project [map_zip_with(cast(struct_map1#x as map,struct>), cast(struct_map2#x as map,struct>), 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] 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 new file mode 100644 index 0000000000000..b6af7ee4ac5ca --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapconcat.sql.out @@ -0,0 +1,246 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( + map(true, false), map(false, true), + map(1Y, 2Y), map(3Y, 4Y), + map(1S, 2S), map(3S, 4S), + map(4, 6), map(7, 8), + map(6L, 7L), map(8L, 9L), + map(9223372036854775809, 9223372036854775808), map(9223372036854775808, 9223372036854775809), + map(1.0D, 2.0D), map(3.0D, 4.0D), + map(float(1.0D), float(2.0D)), map(float(3.0D), float(4.0D)), + map(date '2016-03-14', date '2016-03-13'), map(date '2016-03-12', date '2016-03-11'), + map(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), + map(timestamp '2016-11-11 20:54:00.000', timestamp '2016-11-09 20:54:00.000'), + map('a', 'b'), map('c', 'd'), + map(array('a', 'b'), array('c', 'd')), map(array('e'), array('f')), + map(struct('a', 1), struct('b', 2)), map(struct('c', 3), struct('d', 4)), + map('a', 1), map('c', 2), + map(1, 'a'), map(2, 'c') +) AS various_maps ( + boolean_map1, boolean_map2, + tinyint_map1, tinyint_map2, + smallint_map1, smallint_map2, + int_map1, int_map2, + bigint_map1, bigint_map2, + decimal_map1, decimal_map2, + double_map1, double_map2, + float_map1, float_map2, + date_map1, date_map2, + timestamp_map1, + timestamp_map2, + string_map1, string_map2, + array_map1, array_map2, + struct_map1, struct_map2, + string_int_map1, string_int_map2, + int_string_map1, int_string_map2 +) +-- !query analysis +CreateViewCommand `various_maps`, SELECT * FROM VALUES ( + map(true, false), map(false, true), + map(1Y, 2Y), map(3Y, 4Y), + map(1S, 2S), map(3S, 4S), + map(4, 6), map(7, 8), + map(6L, 7L), map(8L, 9L), + map(9223372036854775809, 9223372036854775808), map(9223372036854775808, 9223372036854775809), + map(1.0D, 2.0D), map(3.0D, 4.0D), + map(float(1.0D), float(2.0D)), map(float(3.0D), float(4.0D)), + map(date '2016-03-14', date '2016-03-13'), map(date '2016-03-12', date '2016-03-11'), + map(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), + map(timestamp '2016-11-11 20:54:00.000', timestamp '2016-11-09 20:54:00.000'), + map('a', 'b'), map('c', 'd'), + map(array('a', 'b'), array('c', 'd')), map(array('e'), array('f')), + map(struct('a', 1), struct('b', 2)), map(struct('c', 3), struct('d', 4)), + map('a', 1), map('c', 2), + map(1, 'a'), map(2, 'c') +) AS various_maps ( + boolean_map1, boolean_map2, + tinyint_map1, tinyint_map2, + smallint_map1, smallint_map2, + int_map1, int_map2, + bigint_map1, bigint_map2, + decimal_map1, decimal_map2, + double_map1, double_map2, + float_map1, float_map2, + date_map1, date_map2, + timestamp_map1, + timestamp_map2, + string_map1, string_map2, + array_map1, array_map2, + struct_map1, struct_map2, + string_int_map1, string_int_map2, + int_string_map1, int_string_map2 +), false, false, LocalTempView, true + +- 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] + + +-- !query +SELECT + map_concat(boolean_map1, boolean_map2) boolean_map, + map_concat(tinyint_map1, tinyint_map2) tinyint_map, + map_concat(smallint_map1, smallint_map2) smallint_map, + map_concat(int_map1, int_map2) int_map, + map_concat(bigint_map1, bigint_map2) bigint_map, + map_concat(decimal_map1, decimal_map2) decimal_map, + map_concat(float_map1, float_map2) float_map, + map_concat(double_map1, double_map2) double_map, + map_concat(date_map1, date_map2) date_map, + map_concat(timestamp_map1, timestamp_map2) timestamp_map, + map_concat(string_map1, string_map2) string_map, + map_concat(array_map1, array_map2) array_map, + map_concat(struct_map1, struct_map2) struct_map, + map_concat(string_int_map1, string_int_map2) string_int_map, + map_concat(int_string_map1, int_string_map2) int_string_map +FROM various_maps +-- !query analysis +Project [map_concat(boolean_map1#x, boolean_map2#x) AS boolean_map#x, map_concat(tinyint_map1#x, tinyint_map2#x) AS tinyint_map#x, map_concat(smallint_map1#x, smallint_map2#x) AS smallint_map#x, map_concat(int_map1#x, int_map2#x) AS int_map#x, map_concat(bigint_map1#x, bigint_map2#x) AS bigint_map#x, map_concat(decimal_map1#x, decimal_map2#x) AS decimal_map#x, map_concat(float_map1#x, float_map2#x) AS float_map#x, map_concat(double_map1#x, double_map2#x) AS double_map#x, map_concat(date_map1#x, date_map2#x) AS date_map#x, map_concat(timestamp_map1#x, timestamp_map2#x) AS timestamp_map#x, map_concat(string_map1#x, string_map2#x) AS string_map#x, map_concat(array_map1#x, array_map2#x) AS array_map#x, map_concat(struct_map1#x, struct_map2#x) AS struct_map#x, map_concat(string_int_map1#x, string_int_map2#x) AS string_int_map#x, map_concat(int_string_map1#x, int_string_map2#x) AS int_string_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,struct_map1#x,struct_map2#x,string_int_map1#x,string_int_map2#x,int_string_map1#x,int_string_map2#x]) + +- 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] + + +-- !query +SELECT + map_concat(tinyint_map1, smallint_map2) ts_map, + map_concat(smallint_map1, int_map2) si_map, + map_concat(int_map1, bigint_map2) ib_map, + map_concat(bigint_map1, decimal_map2) bd_map, + map_concat(decimal_map1, float_map2) df_map, + map_concat(string_map1, date_map2) std_map, + map_concat(timestamp_map1, string_map2) tst_map, + map_concat(string_map1, int_map2) sti_map, + 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,struct_map1#x,struct_map2#x,string_int_map1#x,string_int_map2#x,int_string_map1#x,int_string_map2#x]) + +- 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] + + +-- !query +SELECT + map_concat(tinyint_map1, array_map1) tm_map +FROM various_maps +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"MAP\" or \"MAP, ARRAY>\")", + "functionName" : "`map_concat`", + "sqlExpr" : "\"map_concat(tinyint_map1, array_map1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 47, + "fragment" : "map_concat(tinyint_map1, array_map1)" + } ] +} + + +-- !query +SELECT + map_concat(boolean_map1, int_map2) bi_map +FROM various_maps +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"MAP\" or \"MAP\")", + "functionName" : "`map_concat`", + "sqlExpr" : "\"map_concat(boolean_map1, int_map2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 45, + "fragment" : "map_concat(boolean_map1, int_map2)" + } ] +} + + +-- !query +SELECT + map_concat(int_map1, struct_map2) is_map +FROM various_maps +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"MAP\" or \"MAP, STRUCT>\")", + "functionName" : "`map_concat`", + "sqlExpr" : "\"map_concat(int_map1, struct_map2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 44, + "fragment" : "map_concat(int_map1, struct_map2)" + } ] +} + + +-- !query +SELECT + map_concat(struct_map1, array_map2) ma_map +FROM various_maps +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"MAP, STRUCT>\" or \"MAP, ARRAY>\")", + "functionName" : "`map_concat`", + "sqlExpr" : "\"map_concat(struct_map1, array_map2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 46, + "fragment" : "map_concat(struct_map1, array_map2)" + } ] +} + + +-- !query +SELECT + map_concat(int_map1, array_map2) ms_map +FROM various_maps +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"MAP\" or \"MAP, ARRAY>\")", + "functionName" : "`map_concat`", + "sqlExpr" : "\"map_concat(int_map1, array_map2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 43, + "fragment" : "map_concat(int_map1, array_map2)" + } ] +} 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 new file mode 100644 index 0000000000000..fbfdaa9558b25 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/promoteStrings.sql.out @@ -0,0 +1,3999 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query analysis +CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' + cast(1 as float) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' + cast(1 as double) FROM t +-- !query analysis +Project [(cast(1 as double) + cast(1 as double)) AS (1 + 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 '1' + cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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 + + +-- !query +SELECT '1' + cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(1 + CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "'1' + cast('1' as binary)" + } ] +} + + +-- !query +SELECT '1' + cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(1 + CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "'1' + cast(1 as boolean)" + } ] +} + + +-- !query +SELECT '1' + cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP\"", + "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)" + } ] +} + + +-- !query +SELECT '1' + cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), 1)\"" + }, + "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 analysis +Project [(cast(1 as double) - cast(cast(1 as tinyint) as double)) AS (1 - 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 + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' - cast(1 as float) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' - cast(1 as double) FROM t +-- !query analysis +Project [(cast(1 as double) - cast(1 as double)) AS (1 - 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 '1' - cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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 + + +-- !query +SELECT '1' - cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(1 - CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "'1' - cast('1' as binary)" + } ] +} + + +-- !query +SELECT '1' - cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(1 - CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "'1' - cast(1 as boolean)" + } ] +} + + +-- !query +SELECT '1' - cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "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)" + } ] +} + + +-- !query +SELECT '1' - cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +Project [(cast(1 as date) - cast(2017-12-11 09:30:00 as date)) AS (1 - 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 +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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' * cast(1 as float) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' * cast(1 as double) FROM t +-- !query analysis +Project [(cast(1 as double) * cast(1 as double)) AS (1 * 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 '1' * cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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 + + +-- !query +SELECT '1' * cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(1 * CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "'1' * cast('1' as binary)" + } ] +} + + +-- !query +SELECT '1' * cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(1 * CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "'1' * cast(1 as boolean)" + } ] +} + + +-- !query +SELECT '1' * cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP\"", + "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)" + } ] +} + + +-- !query +SELECT '1' * cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(1 * CAST(2017-12-11 09:30:00 AS 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 analysis +Project [(cast(1 as double) / cast(cast(1 as tinyint) as double)) AS (1 / 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 + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' / cast(1 as float) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' / cast(1 as double) FROM t +-- !query analysis +Project [(cast(1 as double) / cast(1 as double)) AS (1 / 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 '1' / cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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 + + +-- !query +SELECT '1' / cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(1 / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "'1' / cast('1' as binary)" + } ] +} + + +-- !query +SELECT '1' / cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(1 / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "'1' / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT '1' / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP\"", + "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)" + } ] +} + + +-- !query +SELECT '1' / cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(1 / CAST(2017-12-11 09:30:00 AS 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 analysis +Project [(cast(1 as double) % cast(cast(1 as tinyint) as double)) AS (1 % 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 + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' % cast(1 as float) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' % cast(1 as double) FROM t +-- !query analysis +Project [(cast(1 as double) % cast(1 as double)) AS (1 % 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 '1' % cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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 + + +-- !query +SELECT '1' % cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(1 % CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "'1' % cast('1' as binary)" + } ] +} + + +-- !query +SELECT '1' % cast(1 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(1 % CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "'1' % cast(1 as boolean)" + } ] +} + + +-- !query +SELECT '1' % cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP\"", + "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)" + } ] +} + + +-- !query +SELECT '1' % cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(1 % CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' % cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT pmod('1', cast(1 as float)) FROM t +-- !query analysis +Project [pmod(cast(1 as double), cast(cast(1 as float) as double)) AS pmod(1, 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 pmod('1', cast(1 as double)) FROM t +-- !query analysis +Project [pmod(cast(1 as double), cast(1 as double)) AS pmod(1, 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 pmod('1', cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [pmod(cast(1 as double), cast(cast(1 as decimal(10,0)) as double)) AS pmod(1, 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 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 + + +-- !query +SELECT pmod('1', cast('1' as binary)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"pmod(1, CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "pmod('1', cast('1' as binary))" + } ] +} + + +-- !query +SELECT pmod('1', cast(1 as boolean)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"pmod(1, CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "pmod('1', cast(1 as boolean))" + } ] +} + + +-- !query +SELECT pmod('1', cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"pmod(1, CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "pmod('1', cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT pmod('1', cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"pmod(1, CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "pmod('1', cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 float) + '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as double) + '1' FROM t +-- !query analysis +Project [(cast(1 as double) + cast(1 as double)) AS (CAST(1 AS DOUBLE) + 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 decimal(10, 0)) + '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast('1' as binary) + '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) + '1'" + } ] +} + + +-- !query +SELECT cast(1 as boolean) + '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as boolean) + '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DOUBLE\"", + "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'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) + '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(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'" + } ] +} + + +-- !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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 float) - '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as double) - '1' FROM t +-- !query analysis +Project [(cast(1 as double) - cast(1 as double)) AS (CAST(1 AS DOUBLE) - 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 decimal(10, 0)) - '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast('1' as binary) - '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) - 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) - '1'" + } ] +} + + +-- !query +SELECT cast(1 as boolean) - '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) - 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as boolean) - '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "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'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) - '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "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'" + } ] +} + + +-- !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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 float) * '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as double) * '1' FROM t +-- !query analysis +Project [(cast(1 as double) * cast(1 as double)) AS (CAST(1 AS DOUBLE) * 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 decimal(10, 0)) * '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast('1' as binary) * '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) * 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) * '1'" + } ] +} + + +-- !query +SELECT cast(1 as boolean) * '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) * 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as boolean) * '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) * '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DOUBLE\"", + "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'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) * '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(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'" + } ] +} + + +-- !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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 double) / '1' FROM t +-- !query analysis +Project [(cast(1 as double) / cast(1 as double)) AS (CAST(1 AS DOUBLE) / 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 decimal(10, 0)) / '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast('1' as binary) / '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) / '1'" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as boolean) / '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DOUBLE\"", + "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'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(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'" + } ] +} + + +-- !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] ++- 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 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] ++- 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 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] ++- 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 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] ++- 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 float) % '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as double) % '1' FROM t +-- !query analysis +Project [(cast(1 as double) % cast(1 as double)) AS (CAST(1 AS DOUBLE) % 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 decimal(10, 0)) % '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast('1' as binary) % '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) % 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) % '1'" + } ] +} + + +-- !query +SELECT cast(1 as boolean) % '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) % 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as boolean) % '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DOUBLE\"", + "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'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) % '1' FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(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'" + } ] +} + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT pmod(cast(1 as float), '1') FROM t +-- !query analysis +Project [pmod(cast(cast(1 as float) as double), cast(1 as double)) AS pmod(CAST(1 AS FLOAT), 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 pmod(cast(1 as double), '1') FROM t +-- !query analysis +Project [pmod(cast(1 as double), cast(1 as double)) AS pmod(CAST(1 AS DOUBLE), 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 pmod(cast(1 as decimal(10, 0)), '1') FROM t +-- !query analysis +Project [pmod(cast(cast(1 as decimal(10,0)) as double), cast(1 as double)) AS pmod(CAST(1 AS DECIMAL(10,0)), 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 pmod(cast('1' as binary), '1') FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"pmod(CAST(1 AS BINARY), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "pmod(cast('1' as binary), '1')" + } ] +} + + +-- !query +SELECT pmod(cast(1 as boolean), '1') FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"pmod(CAST(1 AS BOOLEAN), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "pmod(cast(1 as boolean), '1')" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), '1') FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "pmod(cast('2017-12-11 09:30:00.0' as timestamp), '1')" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00' as date), '1') FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "pmod(cast('2017-12-11 09:30:00' as date), '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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' = cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as bigint) = cast(1 as bigint)) AS (1 = 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 '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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' = cast(1 as double) FROM t +-- !query analysis +Project [(cast(1 as double) = cast(1 as double)) AS (1 = 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 '1' = cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' = '1' FROM t +-- !query analysis +Project [(1 = 1) 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 + + +-- !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 + + +-- !query +SELECT '1' = cast(1 as boolean) FROM t +-- !query analysis +Project [(cast(1 as boolean) = cast(1 as boolean)) AS (1 = 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 '1' = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +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 +SELECT '1' = cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +Project [(cast(1 as date) = cast(2017-12-11 09:30:00 as date)) AS (1 = 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 +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] ++- 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 smallint) = '1' FROM t +-- !query analysis +Project [(cast(1 as smallint) = cast(1 as smallint)) AS (CAST(1 AS SMALLINT) = 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 int) = '1' FROM t +-- !query analysis +Project [(cast(1 as int) = cast(1 as int)) AS (CAST(1 AS INT) = 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 bigint) = '1' FROM t +-- !query analysis +Project [(cast(1 as bigint) = cast(1 as bigint)) AS (CAST(1 AS BIGINT) = 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 float) = '1' FROM t +-- !query analysis +Project [(cast(1 as float) = cast(1 as float)) AS (CAST(1 AS FLOAT) = 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 double) = '1' FROM t +-- !query analysis +Project [(cast(1 as double) = cast(1 as double)) AS (CAST(1 AS DOUBLE) = 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 decimal(10, 0)) = '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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 + + +-- !query +SELECT cast(1 as boolean) = '1' FROM t +-- !query analysis +Project [(cast(1 as boolean) = cast(1 as boolean)) AS (CAST(1 AS BOOLEAN) = 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.0' as timestamp) = '1' FROM t +-- !query analysis +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 +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 '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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' <=> cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as bigint) <=> cast(1 as bigint)) AS (1 <=> 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 '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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' <=> cast(1 as double) FROM t +-- !query analysis +Project [(cast(1 as double) <=> cast(1 as double)) AS (1 <=> 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 '1' <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' <=> '1' FROM t +-- !query analysis +Project [(1 <=> 1) 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 + + +-- !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 + + +-- !query +SELECT '1' <=> cast(1 as boolean) FROM t +-- !query analysis +Project [(cast(1 as boolean) <=> cast(1 as boolean)) AS (1 <=> 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 '1' <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +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 +SELECT '1' <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +Project [(cast(1 as date) <=> cast(2017-12-11 09:30:00 as date)) AS (1 <=> 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 +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] ++- 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 smallint) <=> '1' FROM t +-- !query analysis +Project [(cast(1 as smallint) <=> cast(1 as smallint)) AS (CAST(1 AS SMALLINT) <=> 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 int) <=> '1' FROM t +-- !query analysis +Project [(cast(1 as int) <=> cast(1 as int)) AS (CAST(1 AS INT) <=> 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 bigint) <=> '1' FROM t +-- !query analysis +Project [(cast(1 as bigint) <=> cast(1 as bigint)) AS (CAST(1 AS BIGINT) <=> 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 float) <=> '1' FROM t +-- !query analysis +Project [(cast(1 as float) <=> cast(1 as float)) AS (CAST(1 AS FLOAT) <=> 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 double) <=> '1' FROM t +-- !query analysis +Project [(cast(1 as double) <=> cast(1 as double)) AS (CAST(1 AS DOUBLE) <=> 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 decimal(10, 0)) <=> '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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 + + +-- !query +SELECT cast(1 as boolean) <=> '1' FROM t +-- !query analysis +Project [(cast(1 as boolean) <=> cast(1 as boolean)) AS (CAST(1 AS BOOLEAN) <=> 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.0' as timestamp) <=> '1' FROM t +-- !query analysis +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 +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 '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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' < cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as bigint) < cast(1 as bigint)) AS (1 < 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 '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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' < cast(1 as double) FROM t +-- !query analysis +Project [(cast(1 as double) < cast(1 as double)) AS (1 < 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 '1' < cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' < '1' FROM t +-- !query analysis +Project [(1 < 1) 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 + + +-- !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 + + +-- !query +SELECT '1' < cast(1 as boolean) FROM t +-- !query analysis +Project [(cast(1 as boolean) < cast(1 as boolean)) AS (1 < 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 '1' < cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +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 +SELECT '1' < cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +Project [(cast(1 as date) < cast(2017-12-11 09:30:00 as date)) AS (1 < 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 +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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' <= cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as bigint) <= cast(1 as bigint)) AS (1 <= 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 '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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' <= cast(1 as double) FROM t +-- !query analysis +Project [(cast(1 as double) <= cast(1 as double)) AS (1 <= 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 '1' <= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' <= '1' FROM t +-- !query analysis +Project [(1 <= 1) 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 + + +-- !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 + + +-- !query +SELECT '1' <= cast(1 as boolean) FROM t +-- !query analysis +Project [(cast(1 as boolean) <= cast(1 as boolean)) AS (1 <= 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 '1' <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +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 +SELECT '1' <= cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +Project [(cast(1 as date) <= cast(2017-12-11 09:30:00 as date)) AS (1 <= 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 +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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' > cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as bigint) > cast(1 as bigint)) AS (1 > 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 '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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' > cast(1 as double) FROM t +-- !query analysis +Project [(cast(1 as double) > cast(1 as double)) AS (1 > 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 '1' > cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' > '1' FROM t +-- !query analysis +Project [(1 > 1) 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 + + +-- !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 + + +-- !query +SELECT '1' > cast(1 as boolean) FROM t +-- !query analysis +Project [(cast(1 as boolean) > cast(1 as boolean)) AS (1 > 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 '1' > cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +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 +SELECT '1' > cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +Project [(cast(1 as date) > cast(2017-12-11 09:30:00 as date)) AS (1 > 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 +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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' >= cast(1 as bigint) FROM t +-- !query analysis +Project [(cast(1 as bigint) >= cast(1 as bigint)) AS (1 >= 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 '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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' >= cast(1 as double) FROM t +-- !query analysis +Project [(cast(1 as double) >= cast(1 as double)) AS (1 >= 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 '1' >= cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' >= '1' FROM t +-- !query analysis +Project [(1 >= 1) 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 + + +-- !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 + + +-- !query +SELECT '1' >= cast(1 as boolean) FROM t +-- !query analysis +Project [(cast(1 as boolean) >= cast(1 as boolean)) AS (1 >= 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 '1' >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +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 +SELECT '1' >= cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +Project [(cast(1 as date) >= cast(2017-12-11 09:30:00 as date)) AS (1 >= 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 +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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' <> cast(1 as bigint) FROM t +-- !query analysis +Project [NOT (cast(1 as bigint) = cast(1 as bigint)) AS (NOT (1 = 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 '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] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' <> cast(1 as double) FROM t +-- !query analysis +Project [NOT (cast(1 as double) = cast(1 as double)) AS (NOT (1 = 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 '1' <> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' <> '1' FROM t +-- !query analysis +Project [NOT (1 = 1) AS (NOT (1 = 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 '1' <> cast('1' as binary) FROM t +-- !query analysis +Project [NOT (cast(1 as binary) = cast(1 as binary)) AS (NOT (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 + + +-- !query +SELECT '1' <> cast(1 as boolean) FROM t +-- !query analysis +Project [NOT (cast(1 as boolean) = cast(1 as boolean)) AS (NOT (1 = 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 '1' <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +Project [NOT (cast(1 as timestamp) = cast(2017-12-11 09:30:00.0 as timestamp)) AS (NOT (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 +SELECT '1' <> cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +Project [NOT (cast(1 as date) = cast(2017-12-11 09:30:00 as date)) AS (NOT (1 = 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 +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] ++- 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 smallint) < '1' FROM t +-- !query analysis +Project [(cast(1 as smallint) < cast(1 as smallint)) AS (CAST(1 AS SMALLINT) < 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 int) < '1' FROM t +-- !query analysis +Project [(cast(1 as int) < cast(1 as int)) AS (CAST(1 AS INT) < 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 bigint) < '1' FROM t +-- !query analysis +Project [(cast(1 as bigint) < cast(1 as bigint)) AS (CAST(1 AS BIGINT) < 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 float) < '1' FROM t +-- !query analysis +Project [(cast(1 as float) < cast(1 as float)) AS (CAST(1 AS FLOAT) < 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 double) < '1' FROM t +-- !query analysis +Project [(cast(1 as double) < cast(1 as double)) AS (CAST(1 AS DOUBLE) < 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 decimal(10, 0)) < '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' < '1' FROM t +-- !query analysis +Project [(1 < 1) 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 + + +-- !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 + + +-- !query +SELECT cast(1 as boolean) < '1' FROM t +-- !query analysis +Project [(cast(1 as boolean) < cast(1 as boolean)) AS (CAST(1 AS BOOLEAN) < 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.0' as timestamp) < '1' FROM t +-- !query analysis +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 +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(1 as tinyint) <= cast(1 as tinyint)) AS (CAST(1 AS TINYINT) <= 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 smallint) <= '1' FROM t +-- !query analysis +Project [(cast(1 as smallint) <= cast(1 as smallint)) AS (CAST(1 AS SMALLINT) <= 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 int) <= '1' FROM t +-- !query analysis +Project [(cast(1 as int) <= cast(1 as int)) AS (CAST(1 AS INT) <= 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 bigint) <= '1' FROM t +-- !query analysis +Project [(cast(1 as bigint) <= cast(1 as bigint)) AS (CAST(1 AS BIGINT) <= 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 float) <= '1' FROM t +-- !query analysis +Project [(cast(1 as float) <= cast(1 as float)) AS (CAST(1 AS FLOAT) <= 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 double) <= '1' FROM t +-- !query analysis +Project [(cast(1 as double) <= cast(1 as double)) AS (CAST(1 AS DOUBLE) <= 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 decimal(10, 0)) <= '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' <= '1' FROM t +-- !query analysis +Project [(1 <= 1) 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 + + +-- !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 + + +-- !query +SELECT cast(1 as boolean) <= '1' FROM t +-- !query analysis +Project [(cast(1 as boolean) <= cast(1 as boolean)) AS (CAST(1 AS BOOLEAN) <= 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.0' as timestamp) <= '1' FROM t +-- !query analysis +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 +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(1 as tinyint) > cast(1 as tinyint)) AS (CAST(1 AS TINYINT) > 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 smallint) > '1' FROM t +-- !query analysis +Project [(cast(1 as smallint) > cast(1 as smallint)) AS (CAST(1 AS SMALLINT) > 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 int) > '1' FROM t +-- !query analysis +Project [(cast(1 as int) > cast(1 as int)) AS (CAST(1 AS INT) > 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 bigint) > '1' FROM t +-- !query analysis +Project [(cast(1 as bigint) > cast(1 as bigint)) AS (CAST(1 AS BIGINT) > 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 float) > '1' FROM t +-- !query analysis +Project [(cast(1 as float) > cast(1 as float)) AS (CAST(1 AS FLOAT) > 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 double) > '1' FROM t +-- !query analysis +Project [(cast(1 as double) > cast(1 as double)) AS (CAST(1 AS DOUBLE) > 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 decimal(10, 0)) > '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' > '1' FROM t +-- !query analysis +Project [(1 > 1) 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 + + +-- !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 + + +-- !query +SELECT cast(1 as boolean) > '1' FROM t +-- !query analysis +Project [(cast(1 as boolean) > cast(1 as boolean)) AS (CAST(1 AS BOOLEAN) > 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.0' as timestamp) > '1' FROM t +-- !query analysis +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 +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(1 as tinyint) >= cast(1 as tinyint)) AS (CAST(1 AS TINYINT) >= 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 smallint) >= '1' FROM t +-- !query analysis +Project [(cast(1 as smallint) >= cast(1 as smallint)) AS (CAST(1 AS SMALLINT) >= 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 int) >= '1' FROM t +-- !query analysis +Project [(cast(1 as int) >= cast(1 as int)) AS (CAST(1 AS INT) >= 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 bigint) >= '1' FROM t +-- !query analysis +Project [(cast(1 as bigint) >= cast(1 as bigint)) AS (CAST(1 AS BIGINT) >= 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 float) >= '1' FROM t +-- !query analysis +Project [(cast(1 as float) >= cast(1 as float)) AS (CAST(1 AS FLOAT) >= 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 double) >= '1' FROM t +-- !query analysis +Project [(cast(1 as double) >= cast(1 as double)) AS (CAST(1 AS DOUBLE) >= 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 decimal(10, 0)) >= '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' >= '1' FROM t +-- !query analysis +Project [(1 >= 1) 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 + + +-- !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 + + +-- !query +SELECT cast(1 as boolean) >= '1' FROM t +-- !query analysis +Project [(cast(1 as boolean) >= cast(1 as boolean)) AS (CAST(1 AS BOOLEAN) >= 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.0' as timestamp) >= '1' FROM t +-- !query analysis +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 +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 [NOT (cast(1 as tinyint) = cast(1 as tinyint)) AS (NOT (CAST(1 AS TINYINT) = 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 smallint) <> '1' FROM t +-- !query analysis +Project [NOT (cast(1 as smallint) = cast(1 as smallint)) AS (NOT (CAST(1 AS SMALLINT) = 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 int) <> '1' FROM t +-- !query analysis +Project [NOT (cast(1 as int) = cast(1 as int)) AS (NOT (CAST(1 AS INT) = 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 bigint) <> '1' FROM t +-- !query analysis +Project [NOT (cast(1 as bigint) = cast(1 as bigint)) AS (NOT (CAST(1 AS BIGINT) = 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 float) <> '1' FROM t +-- !query analysis +Project [NOT (cast(1 as float) = cast(1 as float)) AS (NOT (CAST(1 AS FLOAT) = 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 double) <> '1' FROM t +-- !query analysis +Project [NOT (cast(1 as double) = cast(1 as double)) AS (NOT (CAST(1 AS DOUBLE) = 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 decimal(10, 0)) <> '1' FROM t +-- !query analysis +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] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT '1' <> '1' FROM t +-- !query analysis +Project [NOT (1 = 1) AS (NOT (1 = 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 binary) <> '1' FROM t +-- !query analysis +Project [NOT (cast(1 as binary) = cast(1 as binary)) AS (NOT (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 + + +-- !query +SELECT cast(1 as boolean) <> '1' FROM t +-- !query analysis +Project [NOT (cast(1 as boolean) = cast(1 as boolean)) AS (NOT (CAST(1 AS BOOLEAN) = 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.0' as timestamp) <> '1' FROM t +-- !query analysis +Project [NOT (cast(2017-12-11 09:30:00.0 as timestamp) = cast(1 as timestamp)) AS (NOT (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 +Project [NOT (cast(2017-12-11 09:30:00 as date) = cast(1 as date)) AS (NOT (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 abs('1') FROM t +-- !query analysis +Project [abs(cast(1 as double)) AS abs(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 sum('1') FROM t +-- !query analysis +Aggregate [sum(cast(1 as double)) AS sum(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 avg('1') FROM t +-- !query analysis +Aggregate [avg(cast(1 as double)) AS avg(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 stddev_pop('1') FROM t +-- !query analysis +Aggregate [stddev_pop(cast(1 as double)) AS stddev_pop(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 stddev_samp('1') FROM t +-- !query analysis +Aggregate [stddev_samp(cast(1 as double)) AS stddev_samp(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 - '1' FROM t +-- !query analysis +Project [-cast(1 as double) AS (- 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 + '1' FROM t +-- !query analysis +Project [positive(cast(1 as double)) AS (+ 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 var_pop('1') FROM t +-- !query analysis +Aggregate [var_pop(cast(1 as double)) AS var_pop(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 var_samp('1') FROM t +-- !query analysis +Aggregate [var_samp(cast(1 as double)) AS var_samp(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 skewness('1') FROM t +-- !query analysis +Aggregate [skewness(cast(1 as double)) AS skewness(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 kurtosis('1') FROM t +-- !query analysis +Aggregate [kurtosis(cast(1 as double)) AS kurtosis(1)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation 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 new file mode 100644 index 0000000000000..495f996de279e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/stringCastAndExpressions.sql.out @@ -0,0 +1,378 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 'aa' as a +-- !query analysis +CreateViewCommand `t`, SELECT 'aa' as a, false, false, LocalTempView, true + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select cast(a as byte) from t +-- !query analysis +Project [cast(a#x as tinyint) AS a#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select cast(a as short) from t +-- !query analysis +Project [cast(a#x as smallint) AS a#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select cast(a as int) from t +-- !query analysis +Project [cast(a#x as int) AS a#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select cast(a as long) from t +-- !query analysis +Project [cast(a#x as bigint) AS a#xL] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select cast(a as float) from t +-- !query analysis +Project [cast(a#x as float) AS a#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select cast(a as double) from t +-- !query analysis +Project [cast(a#x as double) AS a#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select cast(a as decimal) from t +-- !query analysis +Project [cast(a#x as decimal(10,0)) AS a#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select cast(a as boolean) from t +-- !query analysis +Project [cast(a#x as boolean) AS a#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select cast(a as timestamp) from t +-- !query analysis +Project [cast(a#x as timestamp) AS a#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select cast(a as date) from t +-- !query analysis +Project [cast(a#x as date) AS a#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select cast(a as binary) from t +-- !query analysis +Project [cast(a#x as binary) AS a#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select cast(a as array) from t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"a\"", + "srcType" : "\"STRING\"", + "targetType" : "\"ARRAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "cast(a as array)" + } ] +} + + +-- !query +select cast(a as struct) from t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"a\"", + "srcType" : "\"STRING\"", + "targetType" : "\"STRUCT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(a as struct)" + } ] +} + + +-- !query +select cast(a as map) from t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"a\"", + "srcType" : "\"STRING\"", + "targetType" : "\"MAP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "cast(a as map)" + } ] +} + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select from_unixtime(a) from t +-- !query analysis +Project [from_unixtime(cast(a#x as bigint), yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles)) AS from_unixtime(a, yyyy-MM-dd HH:mm:ss)#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select from_unixtime('2018-01-01', a) from t +-- !query analysis +Project [from_unixtime(cast(2018-01-01 as bigint), a#x, Some(America/Los_Angeles)) AS from_unixtime(2018-01-01, a)#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !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] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select trunc(a, 'MM') from t +-- !query analysis +Project [trunc(cast(a#x as date), MM) AS trunc(a, MM)#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select trunc('2018-01-01', a) from t +-- !query analysis +Project [trunc(cast(2018-01-01 as date), a#x) AS trunc(2018-01-01, a)#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select unhex('-123') +-- !query analysis +Project [unhex(-123, false) AS unhex(-123)#x] ++- OneRowRelation + + +-- !query +select sha2(a, a) from t +-- !query analysis +Project [sha2(cast(a#x as binary), cast(a#x as int)) AS sha2(a, a)#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select get_json_object(a, a) from t +-- !query analysis +Project [get_json_object(a#x, a#x) AS get_json_object(a, a)#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select json_tuple(a, a) from t +-- !query analysis +Project [c0#x] ++- Generate json_tuple(a#x, a#x), false, [c0#x] + +- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation + + +-- !query +select from_json(a, 'a INT') from t +-- !query analysis +Project [from_json(StructField(a,IntegerType,true), a#x, Some(America/Los_Angeles)) AS from_json(a)#x] ++- SubqueryAlias t + +- View (`t`, [a#x]) + +- Project [cast(a#x as string) AS a#x] + +- Project [aa AS a#x] + +- OneRowRelation 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 new file mode 100644 index 0000000000000..75975258e8831 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/widenSetOperationTypes.sql.out @@ -0,0 +1,3229 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query analysis +CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 tinyint) AS CAST(2 AS TINYINT)#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) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- Project [cast(CAST(1 AS TINYINT)#x as smallint) 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 smallint) AS CAST(2 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 cast(1 as tinyint) FROM t UNION SELECT cast(2 as int) FROM t +-- !query analysis +Distinct ++- Union false, false + :- Project [cast(CAST(1 AS TINYINT)#x as int) 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 int) AS CAST(2 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 cast(1 as tinyint) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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 +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(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 + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as double) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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 +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query analysis +Distinct ++- Union false, false + :- Project [cast(CAST(1 AS TINYINT)#x as decimal(10,0)) 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 decimal(10,0)) AS CAST(2 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 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(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 + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"TINYINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 72, + "fragment" : "SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"TINYINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 71, + "fragment" : "SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"TINYINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 95, + "fragment" : "SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"TINYINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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(CAST(2 AS TINYINT)#x as smallint) AS CAST(2 AS TINYINT)#x] + +- Project [cast(2 as tinyint) AS CAST(2 AS TINYINT)#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 smallint) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 smallint) AS CAST(2 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 cast(1 as smallint) FROM t UNION SELECT cast(2 as int) FROM t +-- !query analysis +Distinct ++- Union false, false + :- Project [cast(CAST(1 AS SMALLINT)#x as int) 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 int) AS CAST(2 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 cast(1 as smallint) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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 +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(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 + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as double) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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 +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query analysis +Distinct ++- Union false, false + :- Project [cast(CAST(1 AS SMALLINT)#x as decimal(10,0)) 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 decimal(10,0)) AS CAST(2 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 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(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 + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"SMALLINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 73, + "fragment" : "SELECT cast(1 as smallint) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"SMALLINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 72, + "fragment" : "SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"SMALLINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 96, + "fragment" : "SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"SMALLINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 89, + "fragment" : "SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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(CAST(2 AS TINYINT)#x as int) AS CAST(2 AS TINYINT)#x] + +- Project [cast(2 as tinyint) AS CAST(2 AS TINYINT)#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 int) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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(CAST(2 AS SMALLINT)#x as int) AS CAST(2 AS SMALLINT)#x] + +- Project [cast(2 as smallint) AS CAST(2 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 cast(1 as int) FROM t UNION SELECT cast(2 as int) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 int) AS CAST(2 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 cast(1 as int) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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 +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(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 + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as double) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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 +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query analysis +Distinct ++- Union false, false + :- Project [cast(CAST(1 AS INT)#x as decimal(10,0)) 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 decimal(10,0)) AS CAST(2 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 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(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 + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 68, + "fragment" : "SELECT cast(1 as int) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 67, + "fragment" : "SELECT cast(1 as int) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as int) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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 +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(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 + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as double) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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 +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query analysis +Distinct ++- Union false, false + :- Project [cast(CAST(1 AS BIGINT)#xL as decimal(20,0)) 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(CAST(2 AS DECIMAL(10,0))#x as decimal(20,0)) 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]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +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 + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"BIGINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 71, + "fragment" : "SELECT cast(1 as bigint) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"BIGINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"BIGINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 94, + "fragment" : "SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"BIGINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +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(2 as tinyint) AS CAST(2 AS TINYINT)#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 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(2 as smallint) AS CAST(2 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 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(2 as int) AS CAST(2 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 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(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 +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as float) 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(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 +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as double) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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 +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +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(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 + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"FLOAT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "SELECT cast(1 as float) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"FLOAT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 69, + "fragment" : "SELECT cast(1 as float) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"FLOAT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 93, + "fragment" : "SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"FLOAT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as int) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as float) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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 +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as double) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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 +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +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 + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"DOUBLE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 71, + "fragment" : "SELECT cast(1 as double) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"DOUBLE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "SELECT cast(1 as double) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"DOUBLE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 94, + "fragment" : "SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"DOUBLE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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(CAST(2 AS TINYINT)#x as decimal(10,0)) AS CAST(2 AS TINYINT)#x] + +- Project [cast(2 as tinyint) AS CAST(2 AS TINYINT)#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 decimal(10, 0)) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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(CAST(2 AS SMALLINT)#x as decimal(10,0)) AS CAST(2 AS SMALLINT)#x] + +- Project [cast(2 as smallint) AS CAST(2 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 cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as int) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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(CAST(2 AS INT)#x as decimal(10,0)) AS CAST(2 AS INT)#x] + +- Project [cast(2 as int) AS CAST(2 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 cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query analysis +Distinct ++- Union false, false + :- Project [cast(CAST(1 AS DECIMAL(10,0))#x as decimal(20,0)) 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(CAST(2 AS BIGINT)#xL as decimal(20,0)) 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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as float) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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(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 +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as double) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 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 +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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 decimal(10,0)) AS CAST(2 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 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(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 + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"DECIMAL(10,0)\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 79, + "fragment" : "SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"DECIMAL(10,0)\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"DECIMAL(10,0)\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 102, + "fragment" : "SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"DECIMAL(10,0)\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 95, + "fragment" : "SELECT cast(1 as decimal(10, 0)) 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(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(2 as tinyint) AS CAST(2 AS TINYINT)#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 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(2 as smallint) AS CAST(2 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 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(2 as int) AS CAST(2 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 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 + + +-- !query +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(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 +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 + + +-- !query +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(2 as decimal(10,0)) AS CAST(2 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 cast(1 as string) FROM t UNION SELECT cast(2 as string) 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(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 +SELECT cast(1 as string) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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" + } ] +} + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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" + } ] +} + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) 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(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 + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) 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(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 + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TINYINT\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 72, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as tinyint) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"SMALLINT\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 73, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as smallint) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as int) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"INT\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 68, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as int) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BIGINT\"", + "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 bigint) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as float) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"FLOAT\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as float) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as double) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DOUBLE\"", + "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 double) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DECIMAL(10,0)\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 79, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as string) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query analysis +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(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 binary) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 72, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 96, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 89, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TINYINT\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 71, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as tinyint) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"SMALLINT\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 72, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as smallint) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as int) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"INT\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 67, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as int) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BIGINT\"", + "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 bigint) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as float) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"FLOAT\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 69, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as float) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as double) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DOUBLE\"", + "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 double) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DECIMAL(10,0)\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as string) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 72, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query analysis +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(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 +SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 95, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TINYINT\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 95, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as tinyint) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"SMALLINT\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 96, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as smallint) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as int) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"INT\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as int) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BIGINT\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "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 bigint) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as float) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"FLOAT\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 93, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as float) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as double) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DOUBLE\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "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 double) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DECIMAL(10,0)\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 102, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as string) FROM t +-- !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 + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 96, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 95, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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(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 +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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(2017-12-11 09:30:00 AS DATE)#x as timestamp) 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 + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TINYINT\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as tinyint) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"SMALLINT\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 89, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as smallint) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as int) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"INT\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as int) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BIGINT\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "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 bigint) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as float) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"FLOAT\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as float) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as double) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DOUBLE\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "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 double) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DECIMAL(10,0)\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 95, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as string) FROM t +-- !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 + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 89, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +Distinct ++- Union false, false + :- Project [cast(CAST(2017-12-12 09:30:00 AS DATE)#x as timestamp) 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(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 +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +Distinct ++- Union false, false + :- 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(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 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 new file mode 100644 index 0000000000000..50ef283061671 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/windowFrameCoercion.sql.out @@ -0,0 +1,378 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query analysis +CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as tinyint)) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS TINYINT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS TINYINT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS TINYINT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as tinyint) ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS TINYINT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL], [1], [cast(1 as tinyint) ASC NULLS FIRST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as smallint)) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS SMALLINT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS SMALLINT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS SMALLINT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as smallint) ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS SMALLINT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL], [1], [cast(1 as smallint) ASC NULLS FIRST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as int)) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS INT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS INT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS INT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as int) ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS INT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL], [1], [cast(1 as int) ASC NULLS FIRST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as bigint)) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BIGINT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BIGINT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BIGINT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as bigint) ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BIGINT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL], [1], [cast(1 as bigint) ASC NULLS FIRST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as float)) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS FLOAT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS FLOAT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS FLOAT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as float) ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS FLOAT) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL], [1], [cast(1 as float) ASC NULLS FIRST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as double)) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DOUBLE) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DOUBLE) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DOUBLE) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as double) ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DOUBLE) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL], [1], [cast(1 as double) ASC NULLS FIRST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as decimal(10, 0))) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DECIMAL(10,0)) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DECIMAL(10,0)) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DECIMAL(10,0)) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as decimal(10,0)) ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DECIMAL(10,0)) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL], [1], [cast(1 as decimal(10,0)) ASC NULLS FIRST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string)) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS STRING) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS STRING) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS STRING) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as string) ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS STRING) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL], [1], [cast(1 as string) ASC NULLS FIRST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary)) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BINARY) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BINARY) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BINARY) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as binary) ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BINARY) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL], [1], [cast(1 as binary) ASC NULLS FIRST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean)) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BOOLEAN) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BOOLEAN) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BOOLEAN) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as boolean) ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BOOLEAN) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL], [1], [cast(1 as boolean) ASC NULLS FIRST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] + +- Window [count(1) windowspecdefinition(1, cast(2017-12-11 09:30:00.0 as timestamp) ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL], [1], [cast(2017-12-11 09:30:00.0 as timestamp) ASC NULLS FIRST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00' as date)) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(2017-12-11 09:30:00 AS DATE) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(2017-12-11 09:30:00 AS DATE) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(2017-12-11 09:30:00 AS DATE) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] + +- Window [count(1) windowspecdefinition(1, cast(2017-12-11 09:30:00 as date) ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(2017-12-11 09:30:00 AS DATE) ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL], [1], [cast(2017-12-11 09:30:00 as date) ASC NULLS FIRST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as tinyint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS TINYINT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS TINYINT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS TINYINT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as tinyint) DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), cast(1 as tinyint))) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS TINYINT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL], [1], [cast(1 as tinyint) DESC NULLS LAST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as smallint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS SMALLINT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS SMALLINT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS SMALLINT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as smallint) DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), cast(1 as smallint))) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS SMALLINT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL], [1], [cast(1 as smallint) DESC NULLS LAST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as int) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS INT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS INT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS INT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as int) DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), 1)) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS INT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL], [1], [cast(1 as int) DESC NULLS LAST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as bigint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BIGINT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BIGINT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BIGINT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as bigint) DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), cast(1 as bigint))) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS BIGINT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL], [1], [cast(1 as bigint) DESC NULLS LAST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as float) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS FLOAT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS FLOAT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS FLOAT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as float) DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), cast(1 as float))) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS FLOAT) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL], [1], [cast(1 as float) DESC NULLS LAST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as double) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DOUBLE) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DOUBLE) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DOUBLE) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as double) DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), cast(1 as double))) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DOUBLE) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL], [1], [cast(1 as double) DESC NULLS LAST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as decimal(10, 0)) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DECIMAL(10,0)) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DECIMAL(10,0)) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DECIMAL(10,0)) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] + +- Window [count(1) windowspecdefinition(1, cast(1 as decimal(10,0)) DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), cast(1 as decimal(10,0)))) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DECIMAL(10,0)) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL], [1], [cast(1 as decimal(10,0)) DESC NULLS LAST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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" : "\"STRING\"", + "location" : "upper", + "sqlExpr" : "\"RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 111, + "fragment" : "(PARTITION BY 1 ORDER BY cast(1 as string) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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" : "\"BINARY\"", + "location" : "upper", + "sqlExpr" : "\"RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 113, + "fragment" : "(PARTITION BY 1 ORDER BY cast('1' as binary) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "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" : "\"BOOLEAN\"", + "location" : "upper", + "sqlExpr" : "\"RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 112, + "fragment" : "(PARTITION BY 1 ORDER BY cast(1 as boolean) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as timestamp) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpecType" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(PARTITION BY 1 ORDER BY CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"", + "valueBoundaryType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 136, + "fragment" : "(PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as timestamp) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00' as date) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query analysis +Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(2017-12-11 09:30:00 AS DATE) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] ++- Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(2017-12-11 09:30:00 AS DATE) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, count(1) OVER (PARTITION BY 1 ORDER BY CAST(2017-12-11 09:30:00 AS DATE) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] + +- Window [count(1) windowspecdefinition(1, cast(2017-12-11 09:30:00 as date) DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), 1)) AS count(1) OVER (PARTITION BY 1 ORDER BY CAST(2017-12-11 09:30:00 AS DATE) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL], [1], [cast(2017-12-11 09:30:00 as date) DESC NULLS LAST] + +- Project + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part1.sql.out new file mode 100644 index 0000000000000..c2ca405645823 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part1.sql.out @@ -0,0 +1,512 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT avg(udf(four)) AS avg_1 FROM onek +-- !query analysis +Aggregate [avg(cast(udf(cast(four#x as string)) as int)) AS avg_1#x] ++- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT udf(avg(a)) AS avg_32 FROM aggtest WHERE a < 100 +-- !query analysis +Aggregate [cast(udf(cast(avg(a#x) as string)) as double) AS avg_32#x] ++- Filter (a#x < 100) + +- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +select CAST(avg(udf(b)) AS Decimal(10,3)) AS avg_107_943 FROM aggtest +-- !query analysis +Aggregate [cast(avg(cast(udf(cast(b#x as string)) as float)) as decimal(10,3)) AS avg_107_943#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT sum(udf(four)) AS sum_1500 FROM onek +-- !query analysis +Aggregate [sum(cast(udf(cast(four#x as string)) as int)) AS sum_1500#xL] ++- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT udf(sum(a)) AS sum_198 FROM aggtest +-- !query analysis +Aggregate [cast(udf(cast(sum(a#x) as string)) as bigint) AS sum_198#xL] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT udf(udf(sum(b))) AS avg_431_773 FROM aggtest +-- !query analysis +Aggregate [cast(udf(cast(cast(udf(cast(sum(b#x) as string)) as double) as string)) as double) AS avg_431_773#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT udf(max(four)) AS max_3 FROM onek +-- !query analysis +Aggregate [cast(udf(cast(max(four#x) as string)) as int) AS max_3#x] ++- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT max(udf(a)) AS max_100 FROM aggtest +-- !query analysis +Aggregate [max(cast(udf(cast(a#x as string)) as int)) AS max_100#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT udf(udf(max(aggtest.b))) AS max_324_78 FROM aggtest +-- !query analysis +Aggregate [cast(udf(cast(cast(udf(cast(max(b#x) as string)) as float) as string)) as float) AS max_324_78#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT stddev_pop(udf(b)) FROM aggtest +-- !query analysis +Aggregate [stddev_pop(cast(cast(udf(cast(b#x as string)) as float) as double)) AS stddev_pop(udf(b))#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT udf(stddev_samp(b)) FROM aggtest +-- !query analysis +Aggregate [cast(udf(cast(stddev_samp(cast(b#x as double)) as string)) as double) AS udf(stddev_samp(b))#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT var_pop(udf(b)) FROM aggtest +-- !query analysis +Aggregate [var_pop(cast(cast(udf(cast(b#x as string)) as float) as double)) AS var_pop(udf(b))#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT udf(var_samp(b)) FROM aggtest +-- !query analysis +Aggregate [cast(udf(cast(var_samp(cast(b#x as double)) as string)) as double) AS udf(var_samp(b))#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT udf(stddev_pop(CAST(b AS Decimal(38,0)))) FROM aggtest +-- !query analysis +Aggregate [cast(udf(cast(stddev_pop(cast(cast(b#x as decimal(38,0)) as double)) as string)) as double) AS udf(stddev_pop(CAST(b AS DECIMAL(38,0))))#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT stddev_samp(CAST(udf(b) AS Decimal(38,0))) FROM aggtest +-- !query analysis +Aggregate [stddev_samp(cast(cast(cast(udf(cast(b#x as string)) as float) as decimal(38,0)) as double)) AS stddev_samp(CAST(udf(b) AS DECIMAL(38,0)))#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT udf(var_pop(CAST(b AS Decimal(38,0)))) FROM aggtest +-- !query analysis +Aggregate [cast(udf(cast(var_pop(cast(cast(b#x as decimal(38,0)) as double)) as string)) as double) AS udf(var_pop(CAST(b AS DECIMAL(38,0))))#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT var_samp(udf(CAST(b AS Decimal(38,0)))) FROM aggtest +-- !query analysis +Aggregate [var_samp(cast(cast(udf(cast(cast(b#x as decimal(38,0)) as string)) as decimal(38,0)) as double)) AS var_samp(udf(CAST(b AS DECIMAL(38,0))))#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT udf(var_pop(1.0)), var_samp(udf(2.0)) +-- !query analysis +Aggregate [cast(udf(cast(var_pop(cast(1.0 as double)) as string)) as double) AS udf(var_pop(1.0))#x, var_samp(cast(cast(udf(cast(2.0 as string)) as decimal(2,1)) as double)) AS var_samp(udf(2.0))#x] ++- OneRowRelation + + +-- !query +SELECT stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))), stddev_samp(CAST(udf(4.0) AS Decimal(38,0))) +-- !query analysis +Aggregate [stddev_pop(cast(cast(udf(cast(cast(3.0 as decimal(38,0)) as string)) as decimal(38,0)) as double)) AS stddev_pop(udf(CAST(3.0 AS DECIMAL(38,0))))#x, stddev_samp(cast(cast(cast(udf(cast(4.0 as string)) as decimal(2,1)) as decimal(38,0)) as double)) AS stddev_samp(CAST(udf(4.0) AS DECIMAL(38,0)))#x] ++- OneRowRelation + + +-- !query +select sum(udf(CAST(null AS int))) from range(1,4) +-- !query analysis +Aggregate [sum(cast(udf(cast(cast(null as int) as string)) as int)) AS sum(udf(CAST(NULL AS INT)))#xL] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select sum(udf(CAST(null AS long))) from range(1,4) +-- !query analysis +Aggregate [sum(cast(udf(cast(cast(null as bigint) as string)) as bigint)) AS sum(udf(CAST(NULL AS BIGINT)))#xL] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4) +-- !query analysis +Aggregate [sum(cast(udf(cast(cast(null as decimal(38,0)) as string)) as decimal(38,0))) AS sum(udf(CAST(NULL AS DECIMAL(38,0))))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select sum(udf(CAST(null AS DOUBLE))) from range(1,4) +-- !query analysis +Aggregate [sum(cast(udf(cast(cast(null as double) as string)) as double)) AS sum(udf(CAST(NULL AS DOUBLE)))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select avg(udf(CAST(null AS int))) from range(1,4) +-- !query analysis +Aggregate [avg(cast(udf(cast(cast(null as int) as string)) as int)) AS avg(udf(CAST(NULL AS INT)))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select avg(udf(CAST(null AS long))) from range(1,4) +-- !query analysis +Aggregate [avg(cast(udf(cast(cast(null as bigint) as string)) as bigint)) AS avg(udf(CAST(NULL AS BIGINT)))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select avg(udf(CAST(null AS Decimal(38,0)))) from range(1,4) +-- !query analysis +Aggregate [avg(cast(udf(cast(cast(null as decimal(38,0)) as string)) as decimal(38,0))) AS avg(udf(CAST(NULL AS DECIMAL(38,0))))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select avg(udf(CAST(null AS DOUBLE))) from range(1,4) +-- !query analysis +Aggregate [avg(cast(udf(cast(cast(null as double) as string)) as double)) AS avg(udf(CAST(NULL AS DOUBLE)))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select sum(CAST(udf('NaN') AS DOUBLE)) from range(1,4) +-- !query analysis +Aggregate [sum(cast(cast(udf(cast(NaN as string)) as string) as double)) AS sum(CAST(udf(NaN) AS DOUBLE))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +select avg(CAST(udf('NaN') AS DOUBLE)) from range(1,4) +-- !query analysis +Aggregate [avg(cast(cast(udf(cast(NaN as string)) as string) as double)) AS avg(CAST(udf(NaN) AS DOUBLE))#x] ++- Range (1, 4, step=1, splits=None) + + +-- !query +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('Infinity'), ('1')) v(x) +-- !query analysis +Aggregate [avg(cast(cast(udf(cast(x#x as string)) as string) as double)) AS avg(CAST(udf(x) AS DOUBLE))#x, var_pop(cast(cast(udf(cast(x#x as string)) as string) as double)) AS var_pop(CAST(udf(x) AS DOUBLE))#x] ++- SubqueryAlias v + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('Infinity'), ('Infinity')) v(x) +-- !query analysis +Aggregate [avg(cast(cast(udf(cast(x#x as string)) as string) as double)) AS avg(CAST(udf(x) AS DOUBLE))#x, var_pop(cast(cast(udf(cast(x#x as string)) as string) as double)) AS var_pop(CAST(udf(x) AS DOUBLE))#x] ++- SubqueryAlias v + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('-Infinity'), ('Infinity')) v(x) +-- !query analysis +Aggregate [avg(cast(cast(udf(cast(x#x as string)) as string) as double)) AS avg(CAST(udf(x) AS DOUBLE))#x, var_pop(cast(cast(udf(cast(x#x as string)) as string) as double)) AS var_pop(CAST(udf(x) AS DOUBLE))#x] ++- SubqueryAlias v + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) +FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x) +-- !query analysis +Aggregate [avg(cast(udf(cast(cast(x#x as double) as string)) as double)) AS avg(udf(CAST(x AS DOUBLE)))#x, cast(udf(cast(var_pop(cast(x#x as double)) as string)) as double) AS udf(var_pop(CAST(x AS DOUBLE)))#x] ++- SubqueryAlias v + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) +FROM (VALUES (7000000000005), (7000000000007)) v(x) +-- !query analysis +Aggregate [avg(cast(udf(cast(cast(x#xL as double) as string)) as double)) AS avg(udf(CAST(x AS DOUBLE)))#x, cast(udf(cast(var_pop(cast(x#xL as double)) as string)) as double) AS udf(var_pop(CAST(x AS DOUBLE)))#x] ++- SubqueryAlias v + +- Project [col1#xL AS x#xL] + +- LocalRelation [col1#xL] + + +-- !query +SELECT regr_count(b, a) FROM aggtest +-- !query analysis +Aggregate [regr_count(b#x, a#x) AS regr_count(b, a)#xL] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT regr_sxx(b, a) FROM aggtest +-- !query analysis +Aggregate [regr_sxx(cast(b#x as double), cast(a#x as double)) AS regr_sxx(b, a)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT regr_syy(b, a) FROM aggtest +-- !query analysis +Aggregate [regr_syy(cast(b#x as double), cast(a#x as double)) AS regr_syy(b, a)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT regr_sxy(b, a) FROM aggtest +-- !query analysis +Aggregate [regr_sxy(cast(b#x as double), cast(a#x as double)) AS regr_sxy(b, a)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT regr_avgx(b, a), regr_avgy(b, a) FROM aggtest +-- !query analysis +Aggregate [regr_avgx(b#x, a#x) AS regr_avgx(b, a)#x, regr_avgy(b#x, a#x) AS regr_avgy(b, a)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT regr_r2(b, a) FROM aggtest +-- !query analysis +Aggregate [regr_r2(cast(b#x as double), cast(a#x as double)) AS regr_r2(b, a)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT regr_slope(b, a), regr_intercept(b, a) FROM aggtest +-- !query analysis +Aggregate [regr_slope(cast(b#x as double), cast(a#x as double)) AS regr_slope(b, a)#x, regr_intercept(cast(b#x as double), cast(a#x as double)) AS regr_intercept(b, a)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT udf(covar_pop(b, udf(a))), covar_samp(udf(b), a) FROM aggtest +-- !query analysis +Aggregate [cast(udf(cast(covar_pop(cast(b#x as double), cast(cast(udf(cast(a#x as string)) as int) as double)) as string)) as double) AS udf(covar_pop(b, udf(a)))#x, covar_samp(cast(cast(udf(cast(b#x as string)) as float) as double), cast(a#x as double)) AS covar_samp(udf(b), a)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +SELECT corr(b, udf(a)) FROM aggtest +-- !query analysis +Aggregate [corr(cast(b#x as double), cast(cast(udf(cast(a#x as string)) as int) as double)) AS corr(b, udf(a))#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +CREATE TEMPORARY VIEW regr_test AS SELECT * FROM VALUES (10,150),(20,250),(30,350),(80,540),(100,200) AS regr_test (x, y) +-- !query analysis +CreateViewCommand `regr_test`, SELECT * FROM VALUES (10,150),(20,250),(30,350),(80,540),(100,200) AS regr_test (x, y), false, false, LocalTempView, true + +- Project [x#x, y#x] + +- SubqueryAlias regr_test + +- LocalRelation [x#x, y#x] + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (10,20,30,80) +-- !query analysis +Aggregate [count(1) AS count(1)#xL, sum(x#x) AS sum(x)#xL, regr_sxx(cast(y#x as double), cast(x#x as double)) AS regr_sxx(y, x)#x, sum(y#x) AS sum(y)#xL, regr_syy(cast(y#x as double), cast(x#x as double)) AS regr_syy(y, x)#x, regr_sxy(cast(y#x as double), cast(x#x as double)) AS regr_sxy(y, x)#x] ++- Filter x#x IN (10,20,30,80) + +- SubqueryAlias regr_test + +- View (`regr_test`, [x#x,y#x]) + +- Project [cast(x#x as int) AS x#x, cast(y#x as int) AS y#x] + +- Project [x#x, y#x] + +- SubqueryAlias regr_test + +- LocalRelation [x#x, y#x] + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test +-- !query analysis +Aggregate [count(1) AS count(1)#xL, sum(x#x) AS sum(x)#xL, regr_sxx(cast(y#x as double), cast(x#x as double)) AS regr_sxx(y, x)#x, sum(y#x) AS sum(y)#xL, regr_syy(cast(y#x as double), cast(x#x as double)) AS regr_syy(y, x)#x, regr_sxy(cast(y#x as double), cast(x#x as double)) AS regr_sxy(y, x)#x] ++- SubqueryAlias regr_test + +- View (`regr_test`, [x#x,y#x]) + +- Project [cast(x#x as int) AS x#x, cast(y#x as int) AS y#x] + +- Project [x#x, y#x] + +- SubqueryAlias regr_test + +- LocalRelation [x#x, y#x] + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (10,20,30) +-- !query analysis +Aggregate [count(1) AS count(1)#xL, sum(x#x) AS sum(x)#xL, regr_sxx(cast(y#x as double), cast(x#x as double)) AS regr_sxx(y, x)#x, sum(y#x) AS sum(y)#xL, regr_syy(cast(y#x as double), cast(x#x as double)) AS regr_syy(y, x)#x, regr_sxy(cast(y#x as double), cast(x#x as double)) AS regr_sxy(y, x)#x] ++- Filter x#x IN (10,20,30) + +- SubqueryAlias regr_test + +- View (`regr_test`, [x#x,y#x]) + +- Project [cast(x#x as int) AS x#x, cast(y#x as int) AS y#x] + +- Project [x#x, y#x] + +- SubqueryAlias regr_test + +- LocalRelation [x#x, y#x] + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (80,100) +-- !query analysis +Aggregate [count(1) AS count(1)#xL, sum(x#x) AS sum(x)#xL, regr_sxx(cast(y#x as double), cast(x#x as double)) AS regr_sxx(y, x)#x, sum(y#x) AS sum(y)#xL, regr_syy(cast(y#x as double), cast(x#x as double)) AS regr_syy(y, x)#x, regr_sxy(cast(y#x as double), cast(x#x as double)) AS regr_sxy(y, x)#x] ++- Filter x#x IN (80,100) + +- SubqueryAlias regr_test + +- View (`regr_test`, [x#x,y#x]) + +- Project [cast(x#x as int) AS x#x, cast(y#x as int) AS y#x] + +- Project [x#x, y#x] + +- SubqueryAlias regr_test + +- LocalRelation [x#x, y#x] + + +-- !query +DROP VIEW regr_test +-- !query analysis +DropTempViewCommand regr_test + + +-- !query +SELECT count(udf(four)) AS cnt_1000 FROM onek +-- !query analysis +Aggregate [count(cast(udf(cast(four#x as string)) as int)) AS cnt_1000#xL] ++- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +SELECT udf(count(DISTINCT four)) AS cnt_4 FROM onek +-- !query analysis +Aggregate [cast(udf(cast(count(distinct four#x) as string)) as bigint) AS cnt_4#xL] ++- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select ten, udf(count(*)), sum(udf(four)) from onek +group by ten order by ten +-- !query analysis +Sort [ten#x ASC NULLS FIRST], true ++- Aggregate [ten#x], [ten#x, cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL, sum(cast(udf(cast(four#x as string)) as int)) AS sum(udf(four))#xL] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select ten, count(udf(four)), udf(sum(DISTINCT four)) from onek +group by ten order by ten +-- !query analysis +Sort [ten#x ASC NULLS FIRST], true ++- Aggregate [ten#x], [ten#x, count(cast(udf(cast(four#x as string)) as int)) AS count(udf(four))#xL, cast(udf(cast(sum(distinct four#x) as string)) as bigint) AS udf(sum(DISTINCT four))#xL] + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select ten, udf(sum(distinct four)) from onek a +group by ten +having exists (select 1 from onek b where udf(sum(distinct a.four)) = b.four) +-- !query analysis +Project [ten#x, udf(sum(DISTINCT four))#xL] ++- Filter exists#x [sum(distinct four#x)#xL] + : +- Project [1 AS 1#x] + : +- Filter (outer(udf(sum(DISTINCT four))#xL) = cast(four#x as bigint)) + : +- SubqueryAlias b + : +- SubqueryAlias spark_catalog.default.onek + : +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- Aggregate [ten#x], [ten#x, cast(udf(cast(sum(distinct four#x) as string)) as bigint) AS udf(sum(DISTINCT four))#xL, sum(distinct four#x) AS sum(distinct four#x)#xL] + +- SubqueryAlias a + +- SubqueryAlias spark_catalog.default.onek + +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b + where sum(distinct a.four + b.four) = udf(b.four)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "sum(DISTINCT (outer(a.four) + b.four))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 114, + "stopIndex" : 142, + "fragment" : "sum(distinct a.four + b.four)" + } ] +} + + +-- !query +select + (select udf(max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)))) +from tenk1 o +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`o`.`unique1`", + "proposal" : "`i`.`unique1`, `i`.`unique2`, `i`.`hundred`, `i`.`even`, `i`.`four`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 75, + "stopIndex" : 83, + "fragment" : "o.unique1" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part2.sql.out new file mode 100644 index 0000000000000..973ed129a4e5f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part2.sql.out @@ -0,0 +1,319 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1) +-- !query analysis +CreateViewCommand `int4_tbl`, select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1), false, false, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias int4_tbl + +- LocalRelation [f1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4) +-- !query analysis +CreateViewCommand `bitwise_test`, SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4), false, true, LocalTempView, true + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0 +-- !query analysis +Aggregate [bit_and(b1#x) AS n1#x, bit_or(b2#x) AS n2#x] ++- Filter (1 = 0) + +- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null +-- !query analysis +Aggregate [bit_and(b4#xL) AS n1#xL, bit_or(b4#xL) AS n2#xL] ++- Filter isnull(b4#xL) + +- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT + BIT_AND(cast(b1 as tinyint)) AS a1, + BIT_AND(cast(b2 as smallint)) AS b1, + BIT_AND(b3) AS c1, + BIT_AND(b4) AS d1, + BIT_OR(cast(b1 as tinyint)) AS e7, + BIT_OR(cast(b2 as smallint)) AS f7, + BIT_OR(b3) AS g7, + BIT_OR(b4) AS h3 +FROM bitwise_test +-- !query analysis +Aggregate [bit_and(cast(b1#x as tinyint)) AS a1#x, bit_and(cast(b2#x as smallint)) AS b1#x, bit_and(b3#x) AS c1#x, bit_and(b4#xL) AS d1#xL, bit_or(cast(b1#x as tinyint)) AS e7#x, bit_or(cast(b2#x as smallint)) AS f7#x, bit_or(b3#x) AS g7#x, bit_or(b4#xL) AS h3#xL] ++- SubqueryAlias bitwise_test + +- View (`bitwise_test`, [b1#x,b2#x,b3#x,b4#xL]) + +- Project [cast(b1#x as int) AS b1#x, cast(b2#x as int) AS b2#x, cast(b3#x as int) AS b3#x, cast(b4#xL as bigint) AS b4#xL] + +- Project [b1#x, b2#x, b3#x, b4#xL] + +- SubqueryAlias bitwise_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#xL] + + +-- !query +SELECT + (NULL AND NULL) IS NULL AS `t`, + (TRUE AND NULL) IS NULL AS `t`, + (FALSE AND NULL) IS NULL AS `t`, + (NULL AND TRUE) IS NULL AS `t`, + (NULL AND FALSE) IS NULL AS `t`, + (TRUE AND TRUE) AS `t`, + NOT (TRUE AND FALSE) AS `t`, + NOT (FALSE AND TRUE) AS `t`, + NOT (FALSE AND FALSE) AS `t` +-- !query analysis +Project [isnull((null AND null)) AS t#x, isnull((true AND cast(null as boolean))) AS t#x, isnull((false AND cast(null as boolean))) AS t#x, isnull((cast(null as boolean) AND true)) AS t#x, isnull((cast(null as boolean) AND false)) AS t#x, (true AND true) AS t#x, NOT (true AND false) AS t#x, NOT (false AND true) AS t#x, NOT (false AND false) AS t#x] ++- OneRowRelation + + +-- !query +SELECT + (NULL OR NULL) IS NULL AS `t`, + (TRUE OR NULL) IS NULL AS `t`, + (FALSE OR NULL) IS NULL AS `t`, + (NULL OR TRUE) IS NULL AS `t`, + (NULL OR FALSE) IS NULL AS `t`, + (TRUE OR TRUE) AS `t`, + (TRUE OR FALSE) AS `t`, + (FALSE OR TRUE) AS `t`, + NOT (FALSE OR FALSE) AS `t` +-- !query analysis +Project [isnull((null OR null)) AS t#x, isnull((true OR cast(null as boolean))) AS t#x, isnull((false OR cast(null as boolean))) AS t#x, isnull((cast(null as boolean) OR true)) AS t#x, isnull((cast(null as boolean) OR false)) AS t#x, (true OR true) AS t#x, (true OR false) AS t#x, (false OR true) AS t#x, NOT (false OR false) AS t#x] ++- OneRowRelation + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4) +-- !query analysis +CreateViewCommand `bool_test`, SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4), false, true, LocalTempView, true + +- Project [b1#x, b2#x, b3#x, b4#x] + +- SubqueryAlias bool_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#x] + + +-- !query +SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0 +-- !query analysis +Aggregate [bool_and(b1#x) AS n1#x, bool_or(b3#x) AS n2#x] ++- Filter (1 = 0) + +- SubqueryAlias bool_test + +- View (`bool_test`, [b1#x,b2#x,b3#x,b4#x]) + +- Project [cast(b1#x as boolean) AS b1#x, cast(b2#x as boolean) AS b2#x, cast(b3#x as boolean) AS b3#x, cast(b4#x as void) AS b4#x] + +- Project [b1#x, b2#x, b3#x, b4#x] + +- SubqueryAlias bool_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#x] + + +-- !query +SELECT + BOOL_AND(b1) AS f1, + BOOL_AND(b2) AS t2, + BOOL_AND(b3) AS f3, + BOOL_AND(b4) AS n4, + BOOL_AND(NOT b2) AS f5, + BOOL_AND(NOT b3) AS t6 +FROM bool_test +-- !query analysis +Aggregate [bool_and(b1#x) AS f1#x, bool_and(b2#x) AS t2#x, bool_and(b3#x) AS f3#x, bool_and(cast(b4#x as boolean)) AS n4#x, bool_and(NOT b2#x) AS f5#x, bool_and(NOT b3#x) AS t6#x] ++- SubqueryAlias bool_test + +- View (`bool_test`, [b1#x,b2#x,b3#x,b4#x]) + +- Project [cast(b1#x as boolean) AS b1#x, cast(b2#x as boolean) AS b2#x, cast(b3#x as boolean) AS b3#x, cast(b4#x as void) AS b4#x] + +- Project [b1#x, b2#x, b3#x, b4#x] + +- SubqueryAlias bool_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#x] + + +-- !query +SELECT + EVERY(b1) AS f1, + EVERY(b2) AS t2, + EVERY(b3) AS f3, + EVERY(b4) AS n4, + EVERY(NOT b2) AS f5, + EVERY(NOT b3) AS t6 +FROM bool_test +-- !query analysis +Aggregate [every(b1#x) AS f1#x, every(b2#x) AS t2#x, every(b3#x) AS f3#x, every(cast(b4#x as boolean)) AS n4#x, every(NOT b2#x) AS f5#x, every(NOT b3#x) AS t6#x] ++- SubqueryAlias bool_test + +- View (`bool_test`, [b1#x,b2#x,b3#x,b4#x]) + +- Project [cast(b1#x as boolean) AS b1#x, cast(b2#x as boolean) AS b2#x, cast(b3#x as boolean) AS b3#x, cast(b4#x as void) AS b4#x] + +- Project [b1#x, b2#x, b3#x, b4#x] + +- SubqueryAlias bool_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#x] + + +-- !query +SELECT + BOOL_OR(b1) AS t1, + BOOL_OR(b2) AS t2, + BOOL_OR(b3) AS f3, + BOOL_OR(b4) AS n4, + BOOL_OR(NOT b2) AS f5, + BOOL_OR(NOT b3) AS t6 +FROM bool_test +-- !query analysis +Aggregate [bool_or(b1#x) AS t1#x, bool_or(b2#x) AS t2#x, bool_or(b3#x) AS f3#x, bool_or(cast(b4#x as boolean)) AS n4#x, bool_or(NOT b2#x) AS f5#x, bool_or(NOT b3#x) AS t6#x] ++- SubqueryAlias bool_test + +- View (`bool_test`, [b1#x,b2#x,b3#x,b4#x]) + +- Project [cast(b1#x as boolean) AS b1#x, cast(b2#x as boolean) AS b2#x, cast(b3#x as boolean) AS b3#x, cast(b4#x as void) AS b4#x] + +- Project [b1#x, b2#x, b3#x, b4#x] + +- SubqueryAlias bool_test + +- LocalRelation [b1#x, b2#x, b3#x, b4#x] + + +-- !query +select min(udf(unique1)) from tenk1 +-- !query analysis +Aggregate [min(cast(udf(cast(unique1#x as string)) as int)) AS min(udf(unique1))#x] ++- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select udf(max(unique1)) from tenk1 +-- !query analysis +Aggregate [cast(udf(cast(max(unique1#x) as string)) as int) AS udf(max(unique1))#x] ++- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select max(unique1) from tenk1 where udf(unique1) < 42 +-- !query analysis +Aggregate [max(unique1#x) AS max(unique1)#x] ++- Filter (cast(udf(cast(unique1#x as string)) as int) < 42) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select max(unique1) from tenk1 where unique1 > udf(42) +-- !query analysis +Aggregate [max(unique1#x) AS max(unique1)#x] ++- Filter (unique1#x > cast(udf(cast(42 as string)) as int)) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select max(unique1) from tenk1 where udf(unique1) > 42000 +-- !query analysis +Aggregate [max(unique1#x) AS max(unique1)#x] ++- Filter (cast(udf(cast(unique1#x as string)) as int) > 42000) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select max(tenthous) from tenk1 where udf(thousand) = 33 +-- !query analysis +Aggregate [max(tenthous#x) AS max(tenthous)#x] ++- Filter (cast(udf(cast(thousand#x as string)) as int) = 33) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select min(tenthous) from tenk1 where udf(thousand) = 33 +-- !query analysis +Aggregate [min(tenthous#x) AS min(tenthous)#x] ++- Filter (cast(udf(cast(thousand#x as string)) as int) = 33) + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select distinct max(udf(unique2)) from tenk1 +-- !query analysis +Distinct ++- Aggregate [max(cast(udf(cast(unique2#x as string)) as int)) AS max(udf(unique2))#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select max(unique2) from tenk1 order by udf(1) +-- !query analysis +Sort [cast(udf(cast(1 as string)) as int) ASC NULLS FIRST], true ++- Aggregate [max(unique2#x) AS max(unique2)#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select max(unique2) from tenk1 order by max(udf(unique2)) +-- !query analysis +Project [max(unique2)#x] ++- Sort [max(cast(udf(cast(unique2#x as string)) as int))#x ASC NULLS FIRST], true + +- Aggregate [max(unique2#x) AS max(unique2)#x, max(cast(udf(cast(unique2#x as string)) as int)) AS max(cast(udf(cast(unique2#x as string)) as int))#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select udf(max(udf(unique2))) from tenk1 order by udf(max(unique2))+1 +-- !query analysis +Project [udf(max(udf(unique2)))#x] ++- Sort [(cast(udf(cast(max(unique2#x)#x as string)) as int) + 1) ASC NULLS FIRST], true + +- Aggregate [cast(udf(cast(max(cast(udf(cast(unique2#x as string)) as int)) as string)) as int) AS udf(max(udf(unique2)))#x, max(unique2#x) AS max(unique2#x)#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select t1.max_unique2, udf(g) from (select max(udf(unique2)) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc +-- !query analysis +Project [max_unique2#x, udf(g)#x] ++- Sort [g#x DESC NULLS LAST], true + +- Project [max_unique2#x, cast(udf(cast(g#x as string)) as int) AS udf(g)#x, g#x] + +- Generate explode(array(1, 2, 3)), false, t2, [g#x] + +- SubqueryAlias t1 + +- Aggregate [max(cast(udf(cast(unique2#x as string)) as int)) AS max_unique2#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select udf(max(100)) from tenk1 +-- !query analysis +Aggregate [cast(udf(cast(max(100) as string)) as int) AS udf(max(100))#x] ++- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part3.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part3.sql.out new file mode 100644 index 0000000000000..2529794a929b6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part3.sql.out @@ -0,0 +1,31 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select udf(max(min(unique1))) from tenk1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NESTED_AGGREGATE_FUNCTION", + "sqlState" : "42607", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 27, + "fragment" : "min(unique1)" + } ] +} + + +-- !query +select udf((select udf(count(*)) + from (values (1)) t0(inner_c))) as col +from (values (2),(3)) t1(outer_c) +-- !query analysis +Project [cast(udf(cast(scalar-subquery#x [] as string)) as bigint) AS col#xL] +: +- Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] +: +- SubqueryAlias t0 +: +- Project [col1#x AS inner_c#x] +: +- LocalRelation [col1#x] ++- SubqueryAlias t1 + +- Project [col1#x AS outer_c#x] + +- LocalRelation [col1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part4.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part4.sql.out new file mode 100644 index 0000000000000..6b33408ba1a63 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part4.sql.out @@ -0,0 +1,31 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select percentile_cont(0.5) within group (order by b) from aggtest +-- !query analysis +Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +select percentile_cont(0.5) within group (order by b), sum(b) from aggtest +-- !query analysis +Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x, sum(b#x) AS sum(b)#x] ++- SubqueryAlias spark_catalog.default.aggtest + +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet + + +-- !query +select percentile_cont(0.5) within group (order by thousand) from tenk1 +-- !query analysis +Aggregate [percentile_cont(thousand#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x] ++- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select percentile_disc(0.5) within group (order by thousand) from tenk1 +-- !query analysis +Aggregate [percentile_disc(thousand#x, cast(0.5 as double), false, 0, 0) AS percentile_disc(0.5) WITHIN GROUP (ORDER BY v)#x] ++- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-case.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-case.sql.out new file mode 100644 index 0000000000000..5502e898ae645 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-case.sql.out @@ -0,0 +1,340 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE CASE_TBL ( + i integer, + f double +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`CASE_TBL`, false + + +-- !query +CREATE TABLE CASE2_TBL ( + i integer, + j integer +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`CASE2_TBL`, false + + +-- !query +INSERT INTO CASE_TBL VALUES (1, 10.1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case_tbl], Append, `spark_catalog`.`default`.`case_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case_tbl), [i, f] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as double) AS f#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE_TBL VALUES (2, 20.2) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case_tbl], Append, `spark_catalog`.`default`.`case_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case_tbl), [i, f] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as double) AS f#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE_TBL VALUES (3, -30.3) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case_tbl], Append, `spark_catalog`.`default`.`case_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case_tbl), [i, f] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as double) AS f#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE_TBL VALUES (4, NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case_tbl], Append, `spark_catalog`.`default`.`case_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case_tbl), [i, f] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as double) AS f#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE2_TBL VALUES (1, -1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE2_TBL VALUES (2, -2) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE2_TBL VALUES (3, -3) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE2_TBL VALUES (2, -4) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE2_TBL VALUES (1, NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO CASE2_TBL VALUES (NULL, -6) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT '3' AS `One`, + CASE + WHEN udf(1 < 2) THEN 3 + END AS `Simple WHEN` +-- !query analysis +Project [3 AS One#x, CASE WHEN cast(udf(cast((1 < 2) as string)) as boolean) THEN 3 END AS Simple WHEN#x] ++- OneRowRelation + + +-- !query +SELECT '' AS `One`, + CASE + WHEN 1 > 2 THEN udf(3) + END AS `Simple default` +-- !query analysis +Project [ AS One#x, CASE WHEN (1 > 2) THEN cast(udf(cast(3 as string)) as int) END AS Simple default#x] ++- OneRowRelation + + +-- !query +SELECT '3' AS `One`, + CASE + WHEN udf(1) < 2 THEN udf(3) + ELSE udf(4) + END AS `Simple ELSE` +-- !query analysis +Project [3 AS One#x, CASE WHEN (cast(udf(cast(1 as string)) as int) < 2) THEN cast(udf(cast(3 as string)) as int) ELSE cast(udf(cast(4 as string)) as int) END AS Simple ELSE#x] ++- OneRowRelation + + +-- !query +SELECT udf('4') AS `One`, + CASE + WHEN 1 > 2 THEN 3 + ELSE 4 + END AS `ELSE default` +-- !query analysis +Project [cast(udf(cast(4 as string)) as string) AS One#x, CASE WHEN (1 > 2) THEN 3 ELSE 4 END AS ELSE default#x] ++- OneRowRelation + + +-- !query +SELECT udf('6') AS `One`, + CASE + WHEN udf(1 > 2) THEN 3 + WHEN udf(4) < 5 THEN 6 + ELSE 7 + END AS `Two WHEN with default` +-- !query analysis +Project [cast(udf(cast(6 as string)) as string) AS One#x, CASE WHEN cast(udf(cast((1 > 2) as string)) as boolean) THEN 3 WHEN (cast(udf(cast(4 as string)) as int) < 5) THEN 6 ELSE 7 END AS Two WHEN with default#x] ++- OneRowRelation + + +-- !query +SELECT '7' AS `None`, + CASE WHEN rand() < udf(0) THEN 1 + END AS `NULL on no matches` +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END +-- !query analysis +Project [CASE WHEN cast(udf(cast((1 = 0) as string)) as boolean) THEN (cast(1 as double) / cast(0 as double)) WHEN (1 = 1) THEN cast(1 as double) ELSE (cast(2 as double) / cast(0 as double)) END AS CASE WHEN udf((1 = 0)) THEN (1 / 0) WHEN (1 = 1) THEN 1 ELSE (2 / 0) END#x] ++- OneRowRelation + + +-- !query +SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END +-- !query analysis +Project [CASE WHEN (1 = 0) THEN (cast(1 as double) / cast(cast(udf(cast(0 as string)) as int) as double)) WHEN (1 = 1) THEN cast(1 as double) ELSE (cast(2 as double) / cast(0 as double)) END AS CASE WHEN (1 = 0) THEN (1 / udf(0)) WHEN (1 = 1) THEN 1 ELSE (2 / 0) END#x] ++- OneRowRelation + + +-- !query +SELECT CASE 'a' WHEN 'a' THEN udf(1) ELSE udf(2) END +-- !query analysis +Project [CASE WHEN (a = a) THEN cast(udf(cast(1 as string)) as int) ELSE cast(udf(cast(2 as string)) as int) END AS CASE WHEN (a = a) THEN udf(1) ELSE udf(2) END#x] ++- OneRowRelation + + +-- !query +SELECT '' AS `Five`, + CASE + WHEN i >= 3 THEN i + END AS `>= 3 or Null` + FROM CASE_TBL +-- !query analysis +Project [ AS Five#x, CASE WHEN (i#x >= 3) THEN i#x END AS >= 3 or Null#x] ++- SubqueryAlias spark_catalog.default.case_tbl + +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + + +-- !query +SELECT '' AS `Five`, + CASE WHEN i >= 3 THEN (i + i) + ELSE i + END AS `Simplest Math` + FROM CASE_TBL +-- !query analysis +Project [ AS Five#x, CASE WHEN (i#x >= 3) THEN (i#x + i#x) ELSE i#x END AS Simplest Math#x] ++- SubqueryAlias spark_catalog.default.case_tbl + +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + + +-- !query +SELECT '' AS `Five`, i AS `Value`, + CASE WHEN (i < 0) THEN 'small' + WHEN (i = 0) THEN 'zero' + WHEN (i = 1) THEN 'one' + WHEN (i = 2) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL +-- !query analysis +Project [ AS Five#x, i#x AS Value#x, CASE WHEN (i#x < 0) THEN small WHEN (i#x = 0) THEN zero WHEN (i#x = 1) THEN one WHEN (i#x = 2) THEN two ELSE big END AS Category#x] ++- SubqueryAlias spark_catalog.default.case_tbl + +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + + +-- !query +SELECT '' AS `Five`, + CASE WHEN ((i < 0) or (i < 0)) THEN 'small' + WHEN ((i = 0) or (i = 0)) THEN 'zero' + WHEN ((i = 1) or (i = 1)) THEN 'one' + WHEN ((i = 2) or (i = 2)) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL +-- !query analysis +Project [ AS Five#x, CASE WHEN ((i#x < 0) OR (i#x < 0)) THEN small WHEN ((i#x = 0) OR (i#x = 0)) THEN zero WHEN ((i#x = 1) OR (i#x = 1)) THEN one WHEN ((i#x = 2) OR (i#x = 2)) THEN two ELSE big END AS Category#x] ++- SubqueryAlias spark_catalog.default.case_tbl + +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + + +-- !query +SELECT * FROM CASE_TBL WHERE udf(COALESCE(f,i)) = 4 +-- !query analysis +Project [i#x, f#x] ++- Filter (cast(udf(cast(coalesce(f#x, cast(i#x as double)) as string)) as double) = cast(4 as double)) + +- SubqueryAlias spark_catalog.default.case_tbl + +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + + +-- !query +SELECT * FROM CASE_TBL WHERE udf(NULLIF(f,i)) = 2 +-- !query analysis +Project [i#x, f#x] ++- Filter (cast(udf(cast(nullif(f#x, i#x) as string)) as double) = cast(2 as double)) + +- SubqueryAlias spark_catalog.default.case_tbl + +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + + +-- !query +SELECT udf(COALESCE(a.f, b.i, b.j)) + FROM CASE_TBL a, CASE2_TBL b +-- !query analysis +Project [cast(udf(cast(coalesce(f#x, cast(i#x as double), cast(j#x as double)) as string)) as double) AS udf(coalesce(f, i, j))#x] ++- Join Inner + :- SubqueryAlias a + : +- SubqueryAlias spark_catalog.default.case_tbl + : +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.case2_tbl + +- Relation spark_catalog.default.case2_tbl[i#x,j#x] parquet + + +-- !query +SELECT * + FROM CASE_TBL a, CASE2_TBL b + WHERE udf(COALESCE(a.f, b.i, b.j)) = 2 +-- !query analysis +Project [i#x, f#x, i#x, j#x] ++- Filter (cast(udf(cast(coalesce(f#x, cast(i#x as double), cast(j#x as double)) as string)) as double) = cast(2 as double)) + +- Join Inner + :- SubqueryAlias a + : +- SubqueryAlias spark_catalog.default.case_tbl + : +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.case2_tbl + +- Relation spark_catalog.default.case2_tbl[i#x,j#x] parquet + + +-- !query +SELECT udf('') AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, + NULLIF(b.i, 4) AS `NULLIF(b.i,4)` + FROM CASE_TBL a, CASE2_TBL b +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS Five#x, nullif(i#x, i#x) AS NULLIF(a.i,b.i)#x, nullif(i#x, 4) AS NULLIF(b.i,4)#x] ++- Join Inner + :- SubqueryAlias a + : +- SubqueryAlias spark_catalog.default.case_tbl + : +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.case2_tbl + +- Relation spark_catalog.default.case2_tbl[i#x,j#x] parquet + + +-- !query +SELECT '' AS `Two`, * + FROM CASE_TBL a, CASE2_TBL b + WHERE udf(COALESCE(f,b.i) = 2) +-- !query analysis +Project [ AS Two#x, i#x, f#x, i#x, j#x] ++- Filter cast(udf(cast((coalesce(f#x, cast(i#x as double)) = cast(2 as double)) as string)) as boolean) + +- Join Inner + :- SubqueryAlias a + : +- SubqueryAlias spark_catalog.default.case_tbl + : +- Relation spark_catalog.default.case_tbl[i#x,f#x] parquet + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.case2_tbl + +- Relation spark_catalog.default.case2_tbl[i#x,j#x] parquet + + +-- !query +SELECT CASE + (CASE vol('bar') + WHEN udf('foo') THEN 'it was foo!' + WHEN udf(vol(null)) THEN 'null input' + WHEN 'bar' THEN 'it was bar!' END + ) + WHEN udf('it was foo!') THEN 'foo recognized' + WHEN 'it was bar!' THEN udf('bar recognized') + ELSE 'unrecognized' END AS col +-- !query analysis +Project [CASE WHEN (CASE WHEN (vol(bar) = cast(udf(cast(foo as string)) as string)) THEN it was foo! WHEN (vol(bar) = cast(udf(cast(vol(cast(null as string)) as string)) as string)) THEN null input WHEN (vol(bar) = bar) THEN it was bar! END = cast(udf(cast(it was foo! as string)) as string)) THEN foo recognized WHEN (CASE WHEN (vol(bar) = cast(udf(cast(foo as string)) as string)) THEN it was foo! WHEN (vol(bar) = cast(udf(cast(vol(cast(null as string)) as string)) as string)) THEN null input WHEN (vol(bar) = bar) THEN it was bar! END = it was bar!) THEN cast(udf(cast(bar recognized as string)) as string) ELSE unrecognized END AS col#x] ++- OneRowRelation + + +-- !query +DROP TABLE CASE_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`CASE_TBL`, false, false, false + + +-- !query +DROP TABLE CASE2_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`CASE2_TBL`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out new file mode 100644 index 0000000000000..3e9cb71d17234 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out @@ -0,0 +1,3565 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT2_TBL(f1) AS VALUES + (smallint(trim('0 '))), + (smallint(trim(' 1234 '))), + (smallint(trim(' -1234'))), + (smallint('32767')), + (smallint('-32767')) +-- !query analysis +CreateViewCommand `INT2_TBL`, [(f1,None)], VALUES + (smallint(trim('0 '))), + (smallint(trim(' 1234 '))), + (smallint(trim(' -1234'))), + (smallint('32767')), + (smallint('-32767')), false, true, LocalTempView, true + +- LocalRelation [col1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM + (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) + AS v(f1) +-- !query analysis +CreateViewCommand `INT4_TBL`, SELECT * FROM + (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) + AS v(f1), false, true, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2) +-- !query analysis +CreateViewCommand `INT8_TBL`, SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2), false, true, LocalTempView, true + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM + (VALUES (0.0), (1004.30), (-34.84), + (cast('1.2345678901234e+200' as double)), (cast('1.2345678901234e-200' as double))) + AS v(f1) +-- !query analysis +CreateViewCommand `FLOAT8_TBL`, SELECT * FROM + (VALUES (0.0), (1004.30), (-34.84), + (cast('1.2345678901234e+200' as double)), (cast('1.2345678901234e-200' as double))) + AS v(f1), false, true, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW TEXT_TBL AS SELECT * FROM + (VALUES ('doh!'), ('hi de ho neighbor')) + AS v(f1) +-- !query analysis +CreateViewCommand `TEXT_TBL`, SELECT * FROM + (VALUES ('doh!'), ('hi de ho neighbor')) + AS v(f1), false, true, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 +-- !query analysis +CreateViewCommand `tenk2`, SELECT * FROM tenk1, false, true, LocalTempView, true + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +CREATE TABLE J1_TBL ( + i integer, + j integer, + t string +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`J1_TBL`, false + + +-- !query +CREATE TABLE J2_TBL ( + i integer, + k integer +) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`J2_TBL`, false + + +-- !query +INSERT INTO J1_TBL VALUES (1, 4, 'one') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (2, 3, 'two') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (3, 2, 'three') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (4, 1, 'four') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (5, 0, 'five') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (6, 6, 'six') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (7, 7, 'seven') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (8, 8, 'eight') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (0, NULL, 'zero') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (NULL, NULL, 'null') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J1_TBL VALUES (NULL, 0, 'zero') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +INSERT INTO J2_TBL VALUES (1, -1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (2, 2) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (3, -3) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (2, 4) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (5, -5) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (5, -5) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (0, NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (NULL, NULL) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO J2_TBL VALUES (NULL, 0) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k] ++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t) + FROM J1_TBL AS tx +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(j#x as string)) as int) AS udf(j)#x, cast(udf(cast(t#x as string)) as string) AS udf(t)#x] ++- SubqueryAlias tx + +- SubqueryAlias spark_catalog.default.j1_tbl + +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + + +-- !query +SELECT udf(udf('')) AS `xxx`, udf(udf(i)), udf(j), udf(t) + FROM J1_TBL tx +-- !query analysis +Project [cast(udf(cast(cast(udf(cast( as string)) as string) as string)) as string) AS xxx#x, cast(udf(cast(cast(udf(cast(i#x as string)) as int) as string)) as int) AS udf(udf(i))#x, cast(udf(cast(j#x as string)) as int) AS udf(j)#x, cast(udf(cast(t#x as string)) as string) AS udf(t)#x] ++- SubqueryAlias tx + +- SubqueryAlias spark_catalog.default.j1_tbl + +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, a, udf(udf(b)), c + FROM J1_TBL AS t1 (a, b, c) +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, a#x, cast(udf(cast(cast(udf(cast(b#x as string)) as int) as string)) as int) AS udf(udf(b))#x, c#x] ++- SubqueryAlias t1 + +- Project [i#x AS a#x, j#x AS b#x, t#x AS c#x] + +- SubqueryAlias spark_catalog.default.j1_tbl + +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(a), udf(b), udf(udf(c)) + FROM J1_TBL t1 (a, b, c) +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(a#x as string)) as int) AS udf(a)#x, cast(udf(cast(b#x as string)) as int) AS udf(b)#x, cast(udf(cast(cast(udf(cast(c#x as string)) as string) as string)) as string) AS udf(udf(c))#x] ++- SubqueryAlias t1 + +- Project [i#x AS a#x, j#x AS b#x, t#x AS c#x] + +- SubqueryAlias spark_catalog.default.j1_tbl + +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(a), b, udf(c), udf(d), e + FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e) +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(a#x as string)) as int) AS udf(a)#x, b#x, cast(udf(cast(c#x as string)) as string) AS udf(c)#x, cast(udf(cast(d#x as string)) as int) AS udf(d)#x, e#x] ++- Join Inner + :- SubqueryAlias t1 + : +- Project [i#x AS a#x, j#x AS b#x, t#x AS c#x] + : +- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias t2 + +- Project [i#x AS d#x, k#x AS e#x] + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, * + FROM J1_TBL CROSS JOIN J2_TBL +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, i#x, j#x, t#x, i#x, k#x] ++- Join Cross + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(i) AS i, udf(k), udf(t) AS t + FROM J1_TBL CROSS JOIN J2_TBL +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`i`", + "referenceNames" : "[`spark_catalog`.`default`.`j1_tbl`.`i`, `spark_catalog`.`default`.`j2_tbl`.`i`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 30, + "fragment" : "i" + } ] +} + + +-- !query +SELECT udf('') AS `xxx`, udf(t1.i) AS i, udf(k), udf(t) + FROM J1_TBL t1 CROSS JOIN J2_TBL t2 +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(i#x as string)) as int) AS i#x, cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(t#x as string)) as string) AS udf(t)#x] ++- Join Cross + :- SubqueryAlias t1 + : +- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias t2 + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf(udf('')) AS `xxx`, udf(udf(ii)) AS ii, udf(udf(tt)) AS tt, udf(udf(kk)) + FROM (J1_TBL CROSS JOIN J2_TBL) + AS tx (ii, jj, tt, ii2, kk) +-- !query analysis +Project [cast(udf(cast(cast(udf(cast( as string)) as string) as string)) as string) AS xxx#x, cast(udf(cast(cast(udf(cast(ii#x as string)) as int) as string)) as int) AS ii#x, cast(udf(cast(cast(udf(cast(tt#x as string)) as string) as string)) as string) AS tt#x, cast(udf(cast(cast(udf(cast(kk#x as string)) as int) as string)) as int) AS udf(udf(kk))#x] ++- SubqueryAlias tx + +- Project [i#x AS ii#x, j#x AS jj#x, t#x AS tt#x, i#x AS ii2#x, k#x AS kk#x] + +- Join Cross + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(udf(j1_tbl.i)), udf(j), udf(t), udf(a.i), udf(a.k), udf(b.i), udf(b.k) + FROM J1_TBL CROSS JOIN J2_TBL a CROSS JOIN J2_TBL b +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(cast(udf(cast(i#x as string)) as int) as string)) as int) AS udf(udf(i))#x, cast(udf(cast(j#x as string)) as int) AS udf(j)#x, cast(udf(cast(t#x as string)) as string) AS udf(t)#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(k#x as string)) as int) AS udf(k)#x] ++- Join Cross + :- Join Cross + : :- SubqueryAlias spark_catalog.default.j1_tbl + : : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + : +- SubqueryAlias a + : +- SubqueryAlias spark_catalog.default.j2_tbl + : +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(i) AS i, udf(j), udf(t) AS t, udf(k) + FROM J1_TBL INNER JOIN J2_TBL USING (i) +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(i#x as string)) as int) AS i#x, cast(udf(cast(j#x as string)) as int) AS udf(j)#x, cast(udf(cast(t#x as string)) as string) AS t#x, cast(udf(cast(k#x as string)) as int) AS udf(k)#x] ++- Project [i#x, j#x, t#x, k#x] + +- Join Inner, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j) AS j, udf(t), udf(k) AS k + FROM J1_TBL JOIN J2_TBL USING (i) +-- !query analysis +Project [cast(udf(cast(cast(udf(cast( as string)) as string) as string)) as string) AS xxx#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(j#x as string)) as int) AS j#x, cast(udf(cast(t#x as string)) as string) AS udf(t)#x, cast(udf(cast(k#x as string)) as int) AS k#x] ++- Project [i#x, j#x, t#x, k#x] + +- Join Inner, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, * + FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, d) USING (a) + ORDER BY udf(udf(a)), udf(d) +-- !query analysis +Sort [cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) ASC NULLS FIRST, cast(udf(cast(d#x as string)) as int) ASC NULLS FIRST], true ++- Project [cast(udf(cast( as string)) as string) AS xxx#x, a#x, b#x, c#x, d#x] + +- Project [a#x, b#x, c#x, d#x] + +- Join Inner, (a#x = a#x) + :- SubqueryAlias t1 + : +- Project [i#x AS a#x, j#x AS b#x, t#x AS c#x] + : +- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias t2 + +- Project [i#x AS a#x, k#x AS d#x] + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL NATURAL JOIN J2_TBL +-- !query analysis +Project [cast(udf(cast(cast(udf(cast( as string)) as string) as string)) as string) AS xxx#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(j#x as string)) as int) AS udf(j)#x, cast(udf(cast(t#x as string)) as string) AS udf(t)#x, cast(udf(cast(k#x as string)) as int) AS udf(k)#x] ++- Project [i#x, j#x, t#x, k#x] + +- Join Inner, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(udf(udf(a))) AS a, udf(b), udf(c), udf(d) + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (a, d) +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) as string)) as int) AS a#x, cast(udf(cast(b#x as string)) as int) AS udf(b)#x, cast(udf(cast(c#x as string)) as string) AS udf(c)#x, cast(udf(cast(d#x as string)) as int) AS udf(d)#x] ++- Project [a#x, b#x, c#x, d#x] + +- Join Inner, (a#x = a#x) + :- SubqueryAlias t1 + : +- Project [i#x AS a#x, j#x AS b#x, t#x AS c#x] + : +- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias t2 + +- Project [i#x AS a#x, k#x AS d#x] + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(udf(a)), udf(udf(b)), udf(udf(c)) AS c, udf(udf(udf(d))) AS d + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (d, a) +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) AS udf(udf(a))#x, cast(udf(cast(cast(udf(cast(b#x as string)) as int) as string)) as int) AS udf(udf(b))#x, cast(udf(cast(cast(udf(cast(c#x as string)) as string) as string)) as string) AS c#x, cast(udf(cast(cast(udf(cast(cast(udf(cast(d#x as string)) as int) as string)) as int) as string)) as int) AS d#x] ++- Project [a#x, b#x, c#x, d#x] + +- Join Inner, (a#x = a#x) + :- SubqueryAlias t1 + : +- Project [i#x AS a#x, j#x AS b#x, t#x AS c#x] + : +- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias t2 + +- Project [i#x AS d#x, k#x AS a#x] + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(udf(J1_TBL.j)), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k) + FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) = J2_TBL.i) +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(cast(udf(cast(j#x as string)) as int) as string)) as int) AS udf(udf(j))#x, cast(udf(cast(t#x as string)) as string) AS udf(t)#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(k#x as string)) as int) AS udf(k)#x] ++- Join Inner, (cast(udf(cast(i#x as string)) as int) = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(udf(J1_TBL.i)), udf(udf(J1_TBL.j)), udf(udf(J1_TBL.t)), J2_TBL.i, J2_TBL.k + FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = udf(J2_TBL.k)) +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(cast(udf(cast(i#x as string)) as int) as string)) as int) AS udf(udf(i))#x, cast(udf(cast(cast(udf(cast(j#x as string)) as int) as string)) as int) AS udf(udf(j))#x, cast(udf(cast(cast(udf(cast(t#x as string)) as string) as string)) as string) AS udf(udf(t))#x, i#x, k#x] ++- Join Inner, (i#x = cast(udf(cast(k#x as string)) as int)) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(J1_TBL.j), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k) + FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) <= udf(udf(J2_TBL.k))) +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(j#x as string)) as int) AS udf(j)#x, cast(udf(cast(t#x as string)) as string) AS udf(t)#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(k#x as string)) as int) AS udf(k)#x] ++- Join Inner, (cast(udf(cast(i#x as string)) as int) <= cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int)) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT OUTER JOIN J2_TBL USING (i) + ORDER BY udf(udf(i)), udf(k), udf(t) +-- !query analysis +Project [xxx#x, udf(i)#x, udf(j)#x, udf(t)#x, udf(k)#x] ++- Sort [cast(udf(cast(cast(udf(cast(i#x as string)) as int) as string)) as int) ASC NULLS FIRST, cast(udf(cast(k#x as string)) as int) ASC NULLS FIRST, cast(udf(cast(t#x as string)) as string) ASC NULLS FIRST], true + +- Project [cast(udf(cast(cast(udf(cast( as string)) as string) as string)) as string) AS xxx#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(j#x as string)) as int) AS udf(j)#x, cast(udf(cast(t#x as string)) as string) AS udf(t)#x, cast(udf(cast(k#x as string)) as int) AS udf(k)#x, i#x, k#x, t#x] + +- Project [i#x, j#x, t#x, k#x] + +- Join LeftOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) + ORDER BY udf(i), udf(udf(k)), udf(t) +-- !query analysis +Project [xxx#x, udf(i)#x, udf(j)#x, udf(t)#x, udf(k)#x] ++- Sort [cast(udf(cast(i#x as string)) as int) ASC NULLS FIRST, cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int) ASC NULLS FIRST, cast(udf(cast(t#x as string)) as string) ASC NULLS FIRST], true + +- Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(j#x as string)) as int) AS udf(j)#x, cast(udf(cast(t#x as string)) as string) AS udf(t)#x, cast(udf(cast(k#x as string)) as int) AS udf(k)#x, i#x, k#x, t#x] + +- Project [i#x, j#x, t#x, k#x] + +- Join LeftOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(udf(i)), udf(j), udf(t), udf(k) + FROM J1_TBL RIGHT OUTER JOIN J2_TBL USING (i) +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(cast(udf(cast(i#x as string)) as int) as string)) as int) AS udf(udf(i))#x, cast(udf(cast(j#x as string)) as int) AS udf(j)#x, cast(udf(cast(t#x as string)) as string) AS udf(t)#x, cast(udf(cast(k#x as string)) as int) AS udf(k)#x] ++- Project [i#x, j#x, t#x, k#x] + +- Join RightOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(i), udf(udf(j)), udf(t), udf(k) + FROM J1_TBL RIGHT JOIN J2_TBL USING (i) +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(cast(udf(cast(j#x as string)) as int) as string)) as int) AS udf(udf(j))#x, cast(udf(cast(t#x as string)) as string) AS udf(t)#x, cast(udf(cast(k#x as string)) as int) AS udf(k)#x] ++- Project [i#x, j#x, t#x, k#x] + +- Join RightOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(udf(t)), udf(k) + FROM J1_TBL FULL OUTER JOIN J2_TBL USING (i) + ORDER BY udf(udf(i)), udf(k), udf(t) +-- !query analysis +Project [xxx#x, udf(i)#x, udf(j)#x, udf(udf(t))#x, udf(k)#x] ++- Sort [cast(udf(cast(cast(udf(cast(i#x as string)) as int) as string)) as int) ASC NULLS FIRST, cast(udf(cast(k#x as string)) as int) ASC NULLS FIRST, cast(udf(cast(t#x as string)) as string) ASC NULLS FIRST], true + +- Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(j#x as string)) as int) AS udf(j)#x, cast(udf(cast(cast(udf(cast(t#x as string)) as string) as string)) as string) AS udf(udf(t))#x, cast(udf(cast(k#x as string)) as int) AS udf(k)#x, i#x, k#x, t#x] + +- Project [coalesce(i#x, i#x) AS i#x, j#x, t#x, k#x] + +- Join FullOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(i), udf(j), t, udf(udf(k)) + FROM J1_TBL FULL JOIN J2_TBL USING (i) + ORDER BY udf(udf(i)), udf(k), udf(udf(t)) +-- !query analysis +Project [xxx#x, udf(i)#x, udf(j)#x, t#x, udf(udf(k))#x] ++- Sort [cast(udf(cast(cast(udf(cast(i#x as string)) as int) as string)) as int) ASC NULLS FIRST, cast(udf(cast(k#x as string)) as int) ASC NULLS FIRST, cast(udf(cast(cast(udf(cast(t#x as string)) as string) as string)) as string) ASC NULLS FIRST], true + +- Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(j#x as string)) as int) AS udf(j)#x, t#x, cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int) AS udf(udf(k))#x, i#x, k#x] + +- Project [coalesce(i#x, i#x) AS i#x, j#x, t#x, k#x] + +- Join FullOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(udf(k)) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(k) = 1) +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(j#x as string)) as int) AS udf(j)#x, cast(udf(cast(t#x as string)) as string) AS udf(t)#x, cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int) AS udf(udf(k))#x] ++- Filter (cast(udf(cast(k#x as string)) as int) = 1) + +- Project [i#x, j#x, t#x, k#x] + +- Join LeftOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(udf(i)) = udf(1)) +-- !query analysis +Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(i#x as string)) as int) AS udf(i)#x, cast(udf(cast(j#x as string)) as int) AS udf(j)#x, cast(udf(cast(t#x as string)) as string) AS udf(t)#x, cast(udf(cast(k#x as string)) as int) AS udf(k)#x] ++- Filter (cast(udf(cast(cast(udf(cast(i#x as string)) as int) as string)) as int) = cast(udf(cast(1 as string)) as int)) + +- Project [i#x, j#x, t#x, k#x] + +- Join LeftOuter, (i#x = i#x) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +CREATE TABLE t1 (name STRING, n INTEGER) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false + + +-- !query +CREATE TABLE t2 (name STRING, n INTEGER) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t2`, false + + +-- !query +CREATE TABLE t3 (name STRING, n INTEGER) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t3`, false + + +-- !query +INSERT INTO t1 VALUES ( 'bb', 11 ) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO t2 VALUES ( 'bb', 12 ) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO t2 VALUES ( 'cc', 22 ) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO t2 VALUES ( 'ee', 42 ) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO t3 VALUES ( 'bb', 13 ) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t3], Append, `spark_catalog`.`default`.`t3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t3), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO t3 VALUES ( 'cc', 23 ) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t3], Append, `spark_catalog`.`default`.`t3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t3), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO t3 VALUES ( 'dd', 33 ) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t3], Append, `spark_catalog`.`default`.`t3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t3), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name) +-- !query analysis +Project [name#x, n#x, n#x, n#x] ++- Project [coalesce(name#x, name#x) AS name#x, n#x, n#x, n#x] + +- Join FullOuter, (name#x = name#x) + :- Project [coalesce(name#x, name#x) AS name#x, n#x, n#x] + : +- Join FullOuter, (name#x = name#x) + : :- SubqueryAlias spark_catalog.default.t1 + : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT udf(name) as name, t2.n FROM t2) as s2 +INNER JOIN +(SELECT udf(udf(name)) as name, t3.n FROM t3) s3 +USING (name) +-- !query analysis +Project [name#x, n#x, n#x] ++- Project [name#x, n#x, n#x] + +- Join Inner, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [cast(udf(cast(name#x as string)) as string) AS name#x, n#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [cast(udf(cast(cast(udf(cast(name#x as string)) as string) as string)) as string) AS name#x, n#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT udf(udf(name)) as name, t2.n FROM t2) as s2 +LEFT JOIN +(SELECT udf(name) as name, t3.n FROM t3) s3 +USING (name) +-- !query analysis +Project [name#x, n#x, n#x] ++- Project [name#x, n#x, n#x] + +- Join LeftOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [cast(udf(cast(cast(udf(cast(name#x as string)) as string) as string)) as string) AS name#x, n#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [cast(udf(cast(name#x as string)) as string) AS name#x, n#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT udf(name), udf(udf(s2.n)), udf(s3.n) FROM +(SELECT * FROM t2) as s2 +FULL JOIN +(SELECT * FROM t3) s3 +USING (name) +-- !query analysis +Project [cast(udf(cast(name#x as string)) as string) AS udf(name)#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS udf(udf(n))#x, cast(udf(cast(n#x as string)) as int) AS udf(n)#x] ++- Project [coalesce(name#x, name#x) AS name#x, n#x, n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [name#x, n#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, n#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT udf(udf(name)) as name, udf(n) as s2_n, udf(2) as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT udf(name) as name, udf(udf(n)) as s3_n, udf(3) as s3_2 FROM t3) s3 +-- !query analysis +Project [name#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] ++- Project [name#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] + +- Join Inner, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [cast(udf(cast(cast(udf(cast(name#x as string)) as string) as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s2_n#x, cast(udf(cast(2 as string)) as int) AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [cast(udf(cast(name#x as string)) as string) AS name#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS s3_n#x, cast(udf(cast(3 as string)) as int) AS s3_2#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT udf(name) as name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL LEFT JOIN +(SELECT udf(udf(name)) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3 +-- !query analysis +Project [name#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] ++- Project [name#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] + +- Join LeftOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [cast(udf(cast(name#x as string)) as string) AS name#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS s2_n#x, 2 AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [cast(udf(cast(cast(udf(cast(name#x as string)) as string) as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s3_n#x, 3 AS s3_2#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT udf(udf(name)) as name, udf(udf(n)) as s3_n, 3 as s3_2 FROM t3) s3 +-- !query analysis +Project [name#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] ++- Project [coalesce(name#x, name#x) AS name#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [cast(udf(cast(name#x as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s2_n#x, 2 AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [cast(udf(cast(cast(udf(cast(name#x as string)) as string) as string)) as string) AS name#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS s3_n#x, 3 AS s3_2#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT udf(udf(name)) as name, udf(n) as s1_n, 1 as s1_1 FROM t1) as s1 +NATURAL INNER JOIN +(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT udf(udf(udf(name))) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3 +-- !query analysis +Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] ++- Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] + +- Join Inner, (name#x = name#x) + :- Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x] + : +- Join Inner, (name#x = name#x) + : :- SubqueryAlias s1 + : : +- Project [cast(udf(cast(cast(udf(cast(name#x as string)) as string) as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s1_n#x, 1 AS s1_1#x] + : : +- SubqueryAlias spark_catalog.default.t1 + : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + : +- SubqueryAlias s2 + : +- Project [cast(udf(cast(name#x as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s2_n#x, 2 AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [cast(udf(cast(cast(udf(cast(cast(udf(cast(name#x as string)) as string) as string)) as string) as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s3_n#x, 3 AS s3_2#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT udf(name) as name, udf(n) as s1_n, udf(udf(1)) as s1_1 FROM t1) as s1 +NATURAL FULL JOIN +(SELECT udf(name) as name, udf(udf(n)) as s2_n, udf(2) as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT udf(udf(name)) as name, udf(n) as s3_n, udf(3) as s3_2 FROM t3) s3 +-- !query analysis +Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] ++- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] + +- Join FullOuter, (name#x = name#x) + :- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x] + : +- Join FullOuter, (name#x = name#x) + : :- SubqueryAlias s1 + : : +- Project [cast(udf(cast(name#x as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s1_n#x, cast(udf(cast(cast(udf(cast(1 as string)) as int) as string)) as int) AS s1_1#x] + : : +- SubqueryAlias spark_catalog.default.t1 + : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + : +- SubqueryAlias s2 + : +- Project [cast(udf(cast(name#x as string)) as string) AS name#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS s2_n#x, cast(udf(cast(2 as string)) as int) AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [cast(udf(cast(cast(udf(cast(name#x as string)) as string) as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s3_n#x, cast(udf(cast(3 as string)) as int) AS s3_2#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT name, udf(udf(s1_n)), udf(s2_n), udf(s3_n) FROM +(SELECT name, udf(udf(n)) as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, udf(n) as s2_n FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, udf(udf(n)) as s3_n FROM t3) as s3 + ) ss2 +-- !query analysis +Project [name#x, cast(udf(cast(cast(udf(cast(s1_n#x as string)) as int) as string)) as int) AS udf(udf(s1_n))#x, cast(udf(cast(s2_n#x as string)) as int) AS udf(s2_n)#x, cast(udf(cast(s3_n#x as string)) as int) AS udf(s3_n)#x] ++- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s2_n#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s1 + : +- Project [name#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS s1_n#x] + : +- SubqueryAlias spark_catalog.default.t1 + : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + +- SubqueryAlias ss2 + +- Project [name#x, s2_n#x, s3_n#x] + +- Project [coalesce(name#x, name#x) AS name#x, s2_n#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [name#x, cast(udf(cast(n#x as string)) as int) AS s2_n#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS s3_n#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT * FROM +(SELECT name, n as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, udf(n) as s3_n FROM t3) as s3 + ) ss2 +-- !query analysis +Project [name#x, s1_n#x, s2_n#x, s2_2#x, s3_n#x] ++- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s2_n#x, s2_2#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s1 + : +- Project [name#x, n#x AS s1_n#x] + : +- SubqueryAlias spark_catalog.default.t1 + : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + +- SubqueryAlias ss2 + +- Project [name#x, s2_n#x, s2_2#x, s3_n#x] + +- Project [coalesce(name#x, name#x) AS name#x, s2_n#x, s2_2#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [name#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS s2_n#x, 2 AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, cast(udf(cast(n#x as string)) as int) AS s3_n#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet + + +-- !query +SELECT s1.name, udf(s1_n), s2.name, udf(udf(s2_n)) FROM + (SELECT name, udf(n) as s1_n FROM t1) as s1 +FULL JOIN + (SELECT name, 2 as s2_n FROM t2) as s2 +ON (udf(udf(s1_n)) = udf(s2_n)) +-- !query analysis +Project [name#x, cast(udf(cast(s1_n#x as string)) as int) AS udf(s1_n)#x, name#x, cast(udf(cast(cast(udf(cast(s2_n#x as string)) as int) as string)) as int) AS udf(udf(s2_n))#x] ++- Join FullOuter, (cast(udf(cast(cast(udf(cast(s1_n#x as string)) as int) as string)) as int) = cast(udf(cast(s2_n#x as string)) as int)) + :- SubqueryAlias s1 + : +- Project [name#x, cast(udf(cast(n#x as string)) as int) AS s1_n#x] + : +- SubqueryAlias spark_catalog.default.t1 + : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + +- SubqueryAlias s2 + +- Project [name#x, 2 AS s2_n#x] + +- SubqueryAlias spark_catalog.default.t2 + +- Relation spark_catalog.default.t2[name#x,n#x] parquet + + +-- !query +create or replace temporary view x as select * from + (values (1,11), (2,22), (3,null), (4,44), (5,null)) + as v(x1, x2) +-- !query analysis +CreateViewCommand `x`, select * from + (values (1,11), (2,22), (3,null), (4,44), (5,null)) + as v(x1, x2), false, true, LocalTempView, true + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +create or replace temporary view y as select * from + (values (1,111), (2,222), (3,333), (4,null)) + as v(y1, y2) +-- !query analysis +CreateViewCommand `y`, select * from + (values (1,111), (2,222), (3,333), (4,null)) + as v(y1, y2), false, true, LocalTempView, true + +- Project [y1#x, y2#x] + +- SubqueryAlias v + +- Project [col1#x AS y1#x, col2#x AS y2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select udf(udf(x1)), udf(x2) from x +-- !query analysis +Project [cast(udf(cast(cast(udf(cast(x1#x as string)) as int) as string)) as int) AS udf(udf(x1))#x, cast(udf(cast(x2#x as string)) as int) AS udf(x2)#x] ++- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select udf(y1), udf(udf(y2)) from y +-- !query analysis +Project [cast(udf(cast(y1#x as string)) as int) AS udf(y1)#x, cast(udf(cast(cast(udf(cast(y2#x as string)) as int) as string)) as int) AS udf(udf(y2))#x] ++- SubqueryAlias y + +- View (`y`, [y1#x,y2#x]) + +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + +- Project [y1#x, y2#x] + +- SubqueryAlias v + +- Project [col1#x AS y1#x, col2#x AS y2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from x left join y on (udf(x1) = udf(udf(y1)) and udf(x2) is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x] ++- Join LeftOuter, ((cast(udf(cast(x1#x as string)) as int) = cast(udf(cast(cast(udf(cast(y1#x as string)) as int) as string)) as int)) AND isnotnull(cast(udf(cast(x2#x as string)) as int))) + :- SubqueryAlias x + : +- View (`x`, [x1#x,x2#x]) + : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : +- Project [x1#x, x2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias y + +- View (`y`, [y1#x,y2#x]) + +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + +- Project [y1#x, y2#x] + +- SubqueryAlias v + +- Project [col1#x AS y1#x, col2#x AS y2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from x left join y on (udf(udf(x1)) = udf(y1) and udf(y2) is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x] ++- Join LeftOuter, ((cast(udf(cast(cast(udf(cast(x1#x as string)) as int) as string)) as int) = cast(udf(cast(y1#x as string)) as int)) AND isnotnull(cast(udf(cast(y2#x as string)) as int))) + :- SubqueryAlias x + : +- View (`x`, [x1#x,x2#x]) + : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : +- Project [x1#x, x2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias y + +- View (`y`, [y1#x,y2#x]) + +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + +- Project [y1#x, y2#x] + +- SubqueryAlias v + +- Project [col1#x AS y1#x, col2#x AS y2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from (x left join y on (udf(x1) = udf(udf(y1)))) left join x xx(xx1,xx2) +on (udf(udf(x1)) = udf(xx1)) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x, xx1#x, xx2#x] ++- Join LeftOuter, (cast(udf(cast(cast(udf(cast(x1#x as string)) as int) as string)) as int) = cast(udf(cast(xx1#x as string)) as int)) + :- Join LeftOuter, (cast(udf(cast(x1#x as string)) as int) = cast(udf(cast(cast(udf(cast(y1#x as string)) as int) as string)) as int)) + : :- SubqueryAlias x + : : +- View (`x`, [x1#x,x2#x]) + : : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : : +- Project [x1#x, x2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias y + : +- View (`y`, [y1#x,y2#x]) + : +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + : +- Project [y1#x, y2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS y1#x, col2#x AS y2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias xx + +- Project [x1#x AS xx1#x, x2#x AS xx2#x] + +- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = xx1 and udf(x2) is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x, xx1#x, xx2#x] ++- Join LeftOuter, ((cast(udf(cast(x1#x as string)) as int) = xx1#x) AND isnotnull(cast(udf(cast(x2#x as string)) as int))) + :- Join LeftOuter, (cast(udf(cast(x1#x as string)) as int) = cast(udf(cast(y1#x as string)) as int)) + : :- SubqueryAlias x + : : +- View (`x`, [x1#x,x2#x]) + : : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : : +- Project [x1#x, x2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias y + : +- View (`y`, [y1#x,y2#x]) + : +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + : +- Project [y1#x, y2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS y1#x, col2#x AS y2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias xx + +- Project [x1#x AS xx1#x, x2#x AS xx2#x] + +- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from (x left join y on (x1 = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = udf(udf(xx1)) and udf(y2) is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x, xx1#x, xx2#x] ++- Join LeftOuter, ((cast(udf(cast(x1#x as string)) as int) = cast(udf(cast(cast(udf(cast(xx1#x as string)) as int) as string)) as int)) AND isnotnull(cast(udf(cast(y2#x as string)) as int))) + :- Join LeftOuter, (x1#x = cast(udf(cast(y1#x as string)) as int)) + : :- SubqueryAlias x + : : +- View (`x`, [x1#x,x2#x]) + : : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : : +- Project [x1#x, x2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias y + : +- View (`y`, [y1#x,y2#x]) + : +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + : +- Project [y1#x, y2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS y1#x, col2#x AS y2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias xx + +- Project [x1#x AS xx1#x, x2#x AS xx2#x] + +- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from (x left join y on (udf(x1) = y1)) left join x xx(xx1,xx2) +on (udf(udf(x1)) = udf(xx1) and udf(udf(xx2)) is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x, xx1#x, xx2#x] ++- Join LeftOuter, ((cast(udf(cast(cast(udf(cast(x1#x as string)) as int) as string)) as int) = cast(udf(cast(xx1#x as string)) as int)) AND isnotnull(cast(udf(cast(cast(udf(cast(xx2#x as string)) as int) as string)) as int))) + :- Join LeftOuter, (cast(udf(cast(x1#x as string)) as int) = y1#x) + : :- SubqueryAlias x + : : +- View (`x`, [x1#x,x2#x]) + : : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : : +- Project [x1#x, x2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias y + : +- View (`y`, [y1#x,y2#x]) + : +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + : +- Project [y1#x, y2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS y1#x, col2#x AS y2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias xx + +- Project [x1#x AS xx1#x, x2#x AS xx2#x] + +- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from (x left join y on (udf(udf(x1)) = udf(udf(y1)))) left join x xx(xx1,xx2) +on (udf(x1) = udf(xx1)) where (udf(x2) is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x, xx1#x, xx2#x] ++- Filter isnotnull(cast(udf(cast(x2#x as string)) as int)) + +- Join LeftOuter, (cast(udf(cast(x1#x as string)) as int) = cast(udf(cast(xx1#x as string)) as int)) + :- Join LeftOuter, (cast(udf(cast(cast(udf(cast(x1#x as string)) as int) as string)) as int) = cast(udf(cast(cast(udf(cast(y1#x as string)) as int) as string)) as int)) + : :- SubqueryAlias x + : : +- View (`x`, [x1#x,x2#x]) + : : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : : +- Project [x1#x, x2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias y + : +- View (`y`, [y1#x,y2#x]) + : +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + : +- Project [y1#x, y2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS y1#x, col2#x AS y2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias xx + +- Project [x1#x AS xx1#x, x2#x AS xx2#x] + +- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = xx1) where (udf(y2) is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x, xx1#x, xx2#x] ++- Filter isnotnull(cast(udf(cast(y2#x as string)) as int)) + +- Join LeftOuter, (cast(udf(cast(x1#x as string)) as int) = xx1#x) + :- Join LeftOuter, (cast(udf(cast(x1#x as string)) as int) = cast(udf(cast(y1#x as string)) as int)) + : :- SubqueryAlias x + : : +- View (`x`, [x1#x,x2#x]) + : : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : : +- Project [x1#x, x2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias y + : +- View (`y`, [y1#x,y2#x]) + : +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + : +- Project [y1#x, y2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS y1#x, col2#x AS y2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias xx + +- Project [x1#x AS xx1#x, x2#x AS xx2#x] + +- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (x1 = udf(xx1)) where (xx2 is not null) +-- !query analysis +Project [x1#x, x2#x, y1#x, y2#x, xx1#x, xx2#x] ++- Filter isnotnull(xx2#x) + +- Join LeftOuter, (x1#x = cast(udf(cast(xx1#x as string)) as int)) + :- Join LeftOuter, (cast(udf(cast(x1#x as string)) as int) = cast(udf(cast(y1#x as string)) as int)) + : :- SubqueryAlias x + : : +- View (`x`, [x1#x,x2#x]) + : : +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + : : +- Project [x1#x, x2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS x1#x, col2#x AS x2#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias y + : +- View (`y`, [y1#x,y2#x]) + : +- Project [cast(y1#x as int) AS y1#x, cast(y2#x as int) AS y2#x] + : +- Project [y1#x, y2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS y1#x, col2#x AS y2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias xx + +- Project [x1#x AS xx1#x, x2#x AS xx2#x] + +- SubqueryAlias x + +- View (`x`, [x1#x,x2#x]) + +- Project [cast(x1#x as int) AS x1#x, cast(x2#x as int) AS x2#x] + +- Project [x1#x, x2#x] + +- SubqueryAlias v + +- Project [col1#x AS x1#x, col2#x AS x2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select udf(udf(count(*))) from tenk1 a where udf(udf(unique1)) in + (select udf(unique1) from tenk1 b join tenk1 c using (unique1) + where udf(udf(b.unique2)) = udf(42)) +-- !query analysis +Aggregate [cast(udf(cast(cast(udf(cast(count(1) as string)) as bigint) as string)) as bigint) AS udf(udf(count(1)))#xL] ++- Filter cast(udf(cast(cast(udf(cast(unique1#x as string)) as int) as string)) as int) IN (list#x []) + : +- Project [cast(udf(cast(unique1#x as string)) as int) AS udf(unique1)#x] + : +- Filter (cast(udf(cast(cast(udf(cast(unique2#x as string)) as int) as string)) as int) = cast(udf(cast(42 as string)) as int)) + : +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, ... 7 more fields] + : +- Join Inner, (unique1#x = unique1#x) + : :- SubqueryAlias b + : : +- SubqueryAlias spark_catalog.default.tenk1 + : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : +- SubqueryAlias c + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias a + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select udf(count(*)) from tenk1 x where + udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and + udf(x.unique1) = 0 and + udf(x.unique1) in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=udf(udf(bb.f1))) +-- !query analysis +Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] ++- Filter ((cast(udf(cast(unique1#x as string)) as int) IN (list#x []) AND (cast(udf(cast(unique1#x as string)) as int) = 0)) AND cast(udf(cast(unique1#x as string)) as int) IN (list#x [])) + : :- Project [cast(udf(cast(f1#x as string)) as int) AS udf(f1)#x] + : : +- Filter (cast(cast(udf(cast(cast(udf(cast(f1#x as string)) as int) as string)) as int) as double) = f1#x) + : : +- Join Inner + : : :- SubqueryAlias a + : : : +- SubqueryAlias int4_tbl + : : : +- View (`INT4_TBL`, [f1#x]) + : : : +- Project [cast(f1#x as int) AS f1#x] + : : : +- Project [f1#x] + : : : +- SubqueryAlias v + : : : +- Project [col1#x AS f1#x] + : : : +- LocalRelation [col1#x] + : : +- SubqueryAlias b + : : +- SubqueryAlias float8_tbl + : : +- View (`FLOAT8_TBL`, [f1#x]) + : : +- Project [cast(f1#x as double) AS f1#x] + : : +- Project [f1#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS f1#x] + : : +- LocalRelation [col1#x] + : +- Project [f1#x] + : +- Filter (cast(f1#x as double) = cast(udf(cast(cast(udf(cast(f1#x as string)) as double) as string)) as double)) + : +- Join Inner + : :- SubqueryAlias aa + : : +- SubqueryAlias int4_tbl + : : +- View (`INT4_TBL`, [f1#x]) + : : +- Project [cast(f1#x as int) AS f1#x] + : : +- Project [f1#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS f1#x] + : : +- LocalRelation [col1#x] + : +- SubqueryAlias bb + : +- SubqueryAlias float8_tbl + : +- View (`FLOAT8_TBL`, [f1#x]) + : +- Project [cast(f1#x as double) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias x + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select udf(udf(count(*))) from tenk1 x where + udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and + udf(x.unique1) = 0 and + udf(udf(x.unique1)) in (select udf(aa.f1) from int4_tbl aa,float8_tbl bb where udf(aa.f1)=udf(udf(bb.f1))) +-- !query analysis +Aggregate [cast(udf(cast(cast(udf(cast(count(1) as string)) as bigint) as string)) as bigint) AS udf(udf(count(1)))#xL] ++- Filter ((cast(udf(cast(unique1#x as string)) as int) IN (list#x []) AND (cast(udf(cast(unique1#x as string)) as int) = 0)) AND cast(udf(cast(cast(udf(cast(unique1#x as string)) as int) as string)) as int) IN (list#x [])) + : :- Project [cast(udf(cast(f1#x as string)) as int) AS udf(f1)#x] + : : +- Filter (cast(cast(udf(cast(cast(udf(cast(f1#x as string)) as int) as string)) as int) as double) = f1#x) + : : +- Join Inner + : : :- SubqueryAlias a + : : : +- SubqueryAlias int4_tbl + : : : +- View (`INT4_TBL`, [f1#x]) + : : : +- Project [cast(f1#x as int) AS f1#x] + : : : +- Project [f1#x] + : : : +- SubqueryAlias v + : : : +- Project [col1#x AS f1#x] + : : : +- LocalRelation [col1#x] + : : +- SubqueryAlias b + : : +- SubqueryAlias float8_tbl + : : +- View (`FLOAT8_TBL`, [f1#x]) + : : +- Project [cast(f1#x as double) AS f1#x] + : : +- Project [f1#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS f1#x] + : : +- LocalRelation [col1#x] + : +- Project [cast(udf(cast(f1#x as string)) as int) AS udf(f1)#x] + : +- Filter (cast(cast(udf(cast(f1#x as string)) as int) as double) = cast(udf(cast(cast(udf(cast(f1#x as string)) as double) as string)) as double)) + : +- Join Inner + : :- SubqueryAlias aa + : : +- SubqueryAlias int4_tbl + : : +- View (`INT4_TBL`, [f1#x]) + : : +- Project [cast(f1#x as int) AS f1#x] + : : +- Project [f1#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS f1#x] + : : +- LocalRelation [col1#x] + : +- SubqueryAlias bb + : +- SubqueryAlias float8_tbl + : +- View (`FLOAT8_TBL`, [f1#x]) + : +- Project [cast(f1#x as double) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias x + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select * from int8_tbl i1 left join (int8_tbl i2 join + (select udf(123) as x) ss on udf(udf(i2.q1)) = udf(x)) on udf(udf(i1.q2)) = udf(udf(i2.q2)) +order by udf(udf(1)), 2 +-- !query analysis +Sort [cast(udf(cast(cast(udf(cast(1 as string)) as int) as string)) as int) ASC NULLS FIRST, q2#xL ASC NULLS FIRST], true ++- Project [q1#xL, q2#xL, q1#xL, q2#xL, x#x] + +- Join LeftOuter, (cast(udf(cast(cast(udf(cast(q2#xL as string)) as bigint) as string)) as bigint) = cast(udf(cast(cast(udf(cast(q2#xL as string)) as bigint) as string)) as bigint)) + :- SubqueryAlias i1 + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- Join Inner, (cast(udf(cast(cast(udf(cast(q1#xL as string)) as bigint) as string)) as bigint) = cast(cast(udf(cast(x#x as string)) as int) as bigint)) + :- SubqueryAlias i2 + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias ss + +- Project [cast(udf(cast(123 as string)) as int) AS x#x] + +- OneRowRelation + + +-- !query +select udf(count(*)) +from + (select udf(t3.tenthous) as x1, udf(coalesce(udf(t1.stringu1), udf(t2.stringu1))) as x2 + from tenk1 t1 + left join tenk1 t2 on udf(t1.unique1) = udf(t2.unique1) + join tenk1 t3 on t1.unique2 = udf(t3.unique2)) ss, + tenk1 t4, + tenk1 t5 +where udf(t4.thousand) = udf(t5.unique1) and udf(udf(ss.x1)) = t4.tenthous and udf(ss.x2) = udf(udf(t5.stringu1)) +-- !query analysis +Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] ++- Filter (((cast(udf(cast(thousand#x as string)) as int) = cast(udf(cast(unique1#x as string)) as int)) AND (cast(udf(cast(cast(udf(cast(x1#x as string)) as int) as string)) as int) = tenthous#x)) AND (cast(udf(cast(x2#x as string)) as string) = cast(udf(cast(cast(udf(cast(stringu1#x as string)) as string) as string)) as string))) + +- Join Inner + :- Join Inner + : :- SubqueryAlias ss + : : +- Project [cast(udf(cast(tenthous#x as string)) as int) AS x1#x, cast(udf(cast(coalesce(cast(udf(cast(stringu1#x as string)) as string), cast(udf(cast(stringu1#x as string)) as string)) as string)) as string) AS x2#x] + : : +- Join Inner, (unique2#x = cast(udf(cast(unique2#x as string)) as int)) + : : :- Join LeftOuter, (cast(udf(cast(unique1#x as string)) as int) = cast(udf(cast(unique1#x as string)) as int)) + : : : :- SubqueryAlias t1 + : : : : +- SubqueryAlias spark_catalog.default.tenk1 + : : : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : : : +- SubqueryAlias t2 + : : : +- SubqueryAlias spark_catalog.default.tenk1 + : : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : : +- SubqueryAlias t3 + : : +- SubqueryAlias spark_catalog.default.tenk1 + : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : +- SubqueryAlias t4 + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias t5 + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select udf(a.f1), udf(b.f1), udf(t.thousand), udf(t.tenthous) from + tenk1 t, + (select udf(udf(sum(udf(f1))+1)) as f1 from int4_tbl i4a) a, + (select udf(sum(udf(f1))) as f1 from int4_tbl i4b) b +where b.f1 = udf(t.thousand) and udf(a.f1) = udf(b.f1) and udf((udf(a.f1)+udf(b.f1)+999)) = udf(udf(t.tenthous)) +-- !query analysis +Project [cast(udf(cast(f1#xL as string)) as bigint) AS udf(f1)#xL, cast(udf(cast(f1#xL as string)) as bigint) AS udf(f1)#xL, cast(udf(cast(thousand#x as string)) as int) AS udf(thousand)#x, cast(udf(cast(tenthous#x as string)) as int) AS udf(tenthous)#x] ++- Filter (((f1#xL = cast(cast(udf(cast(thousand#x as string)) as int) as bigint)) AND (cast(udf(cast(f1#xL as string)) as bigint) = cast(udf(cast(f1#xL as string)) as bigint))) AND (cast(udf(cast(((cast(udf(cast(f1#xL as string)) as bigint) + cast(udf(cast(f1#xL as string)) as bigint)) + cast(999 as bigint)) as string)) as bigint) = cast(cast(udf(cast(cast(udf(cast(tenthous#x as string)) as int) as string)) as int) as bigint))) + +- Join Inner + :- Join Inner + : :- SubqueryAlias t + : : +- SubqueryAlias spark_catalog.default.tenk1 + : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : +- SubqueryAlias a + : +- Aggregate [cast(udf(cast(cast(udf(cast((sum(cast(udf(cast(f1#x as string)) as int)) + cast(1 as bigint)) as string)) as bigint) as string)) as bigint) AS f1#xL] + : +- SubqueryAlias i4a + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias b + +- Aggregate [cast(udf(cast(sum(cast(udf(cast(f1#x as string)) as int)) as string)) as bigint) AS f1#xL] + +- SubqueryAlias i4b + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from + j1_tbl full join + (select * from j2_tbl order by udf(udf(j2_tbl.i)) desc, udf(j2_tbl.k) asc) j2_tbl + on udf(j1_tbl.i) = udf(j2_tbl.i) and udf(j1_tbl.i) = udf(j2_tbl.k) +-- !query analysis +Project [i#x, j#x, t#x, i#x, k#x] ++- Join FullOuter, ((cast(udf(cast(i#x as string)) as int) = cast(udf(cast(i#x as string)) as int)) AND (cast(udf(cast(i#x as string)) as int) = cast(udf(cast(k#x as string)) as int))) + :- SubqueryAlias spark_catalog.default.j1_tbl + : +- Relation spark_catalog.default.j1_tbl[i#x,j#x,t#x] parquet + +- SubqueryAlias j2_tbl + +- Sort [cast(udf(cast(cast(udf(cast(i#x as string)) as int) as string)) as int) DESC NULLS LAST, cast(udf(cast(k#x as string)) as int) ASC NULLS FIRST], true + +- Project [i#x, k#x] + +- SubqueryAlias spark_catalog.default.j2_tbl + +- Relation spark_catalog.default.j2_tbl[i#x,k#x] parquet + + +-- !query +select udf(count(*)) from + (select * from tenk1 x order by udf(x.thousand), udf(udf(x.twothousand)), x.fivethous) x + left join + (select * from tenk1 y order by udf(y.unique2)) y + on udf(x.thousand) = y.unique2 and x.twothousand = udf(y.hundred) and x.fivethous = y.unique2 +-- !query analysis +Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] ++- Join LeftOuter, (((cast(udf(cast(thousand#x as string)) as int) = unique2#x) AND (twothousand#x = cast(udf(cast(hundred#x as string)) as int))) AND (fivethous#x = unique2#x)) + :- SubqueryAlias x + : +- Sort [cast(udf(cast(thousand#x as string)) as int) ASC NULLS FIRST, cast(udf(cast(cast(udf(cast(twothousand#x as string)) as int) as string)) as int) ASC NULLS FIRST, fivethous#x ASC NULLS FIRST], true + : +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + : +- SubqueryAlias x + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias y + +- Sort [cast(udf(cast(unique2#x as string)) as int) ASC NULLS FIRST], true + +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x] + +- SubqueryAlias y + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +DROP TABLE t1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t1`, false, false, false + + +-- !query +DROP TABLE t2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t2`, false, false, false + + +-- !query +DROP TABLE t3 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t3`, false, false, false + + +-- !query +DROP TABLE J1_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`J1_TBL`, false, false, false + + +-- !query +DROP TABLE J2_TBL +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`J2_TBL`, false, false, false + + +-- !query +create or replace temporary view tt1 as select * from + (values (1, 11), (2, NULL)) + as v(tt1_id, joincol) +-- !query analysis +CreateViewCommand `tt1`, select * from + (values (1, 11), (2, NULL)) + as v(tt1_id, joincol), false, true, LocalTempView, true + +- Project [tt1_id#x, joincol#x] + +- SubqueryAlias v + +- Project [col1#x AS tt1_id#x, col2#x AS joincol#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +create or replace temporary view tt2 as select * from + (values (21, 11), (22, 11)) + as v(tt2_id, joincol) +-- !query analysis +CreateViewCommand `tt2`, select * from + (values (21, 11), (22, 11)) + as v(tt2_id, joincol), false, true, LocalTempView, true + +- Project [tt2_id#x, joincol#x] + +- SubqueryAlias v + +- Project [col1#x AS tt2_id#x, col2#x AS joincol#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select tt1.*, tt2.* from tt1 left join tt2 on udf(udf(tt1.joincol)) = udf(tt2.joincol) +-- !query analysis +Project [tt1_id#x, joincol#x, tt2_id#x, joincol#x] ++- Join LeftOuter, (cast(udf(cast(cast(udf(cast(joincol#x as string)) as int) as string)) as int) = cast(udf(cast(joincol#x as string)) as int)) + :- SubqueryAlias tt1 + : +- View (`tt1`, [tt1_id#x,joincol#x]) + : +- Project [cast(tt1_id#x as int) AS tt1_id#x, cast(joincol#x as int) AS joincol#x] + : +- Project [tt1_id#x, joincol#x] + : +- SubqueryAlias v + : +- Project [col1#x AS tt1_id#x, col2#x AS joincol#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias tt2 + +- View (`tt2`, [tt2_id#x,joincol#x]) + +- Project [cast(tt2_id#x as int) AS tt2_id#x, cast(joincol#x as int) AS joincol#x] + +- Project [tt2_id#x, joincol#x] + +- SubqueryAlias v + +- Project [col1#x AS tt2_id#x, col2#x AS joincol#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select tt1.*, tt2.* from tt2 right join tt1 on udf(udf(tt1.joincol)) = udf(udf(tt2.joincol)) +-- !query analysis +Project [tt1_id#x, joincol#x, tt2_id#x, joincol#x] ++- Join RightOuter, (cast(udf(cast(cast(udf(cast(joincol#x as string)) as int) as string)) as int) = cast(udf(cast(cast(udf(cast(joincol#x as string)) as int) as string)) as int)) + :- SubqueryAlias tt2 + : +- View (`tt2`, [tt2_id#x,joincol#x]) + : +- Project [cast(tt2_id#x as int) AS tt2_id#x, cast(joincol#x as int) AS joincol#x] + : +- Project [tt2_id#x, joincol#x] + : +- SubqueryAlias v + : +- Project [col1#x AS tt2_id#x, col2#x AS joincol#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias tt1 + +- View (`tt1`, [tt1_id#x,joincol#x]) + +- Project [cast(tt1_id#x as int) AS tt1_id#x, cast(joincol#x as int) AS joincol#x] + +- Project [tt1_id#x, joincol#x] + +- SubqueryAlias v + +- Project [col1#x AS tt1_id#x, col2#x AS joincol#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select udf(count(*)) from tenk1 a, tenk1 b + where udf(a.hundred) = b.thousand and udf(udf((b.fivethous % 10)) < 10) +-- !query analysis +Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] ++- Filter ((cast(udf(cast(hundred#x as string)) as int) = thousand#x) AND cast(udf(cast((cast(udf(cast((fivethous#x % 10) as string)) as int) < 10) as string)) as boolean)) + +- Join Inner + :- SubqueryAlias a + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +DROP TABLE IF EXISTS tt3 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tt3`, true, false, false + + +-- !query +CREATE TABLE tt3(f1 int, f2 string) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tt3`, false + + +-- !query +INSERT INTO tt3 SELECT x.id, repeat('xyzzy', 100) FROM range(1,10001) x +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tt3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/tt3], Append, `spark_catalog`.`default`.`tt3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tt3), [f1, f2] ++- Project [cast(id#xL as int) AS f1#x, cast(repeat(xyzzy, 100)#x as string) AS f2#x] + +- Project [id#xL, repeat(xyzzy, 100) AS repeat(xyzzy, 100)#x] + +- SubqueryAlias x + +- Range (1, 10001, step=1, splits=None) + + +-- !query +DROP TABLE IF EXISTS tt4 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tt4`, true, false, false + + +-- !query +CREATE TABLE tt4(f1 int) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`tt4`, false + + +-- !query +INSERT INTO tt4 VALUES (0),(1),(9999) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tt4, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/tt4], Append, `spark_catalog`.`default`.`tt4`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tt4), [f1] ++- Project [cast(col1#x as int) AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT udf(udf(a.f1)) as f1 +FROM tt4 a +LEFT JOIN ( + SELECT b.f1 + FROM tt3 b LEFT JOIN tt3 c ON udf(b.f1) = udf(c.f1) + WHERE udf(c.f1) IS NULL +) AS d ON udf(a.f1) = d.f1 +WHERE udf(udf(d.f1)) IS NULL +-- !query analysis +Project [cast(udf(cast(cast(udf(cast(f1#x as string)) as int) as string)) as int) AS f1#x] ++- Filter isnull(cast(udf(cast(cast(udf(cast(f1#x as string)) as int) as string)) as int)) + +- Join LeftOuter, (cast(udf(cast(f1#x as string)) as int) = f1#x) + :- SubqueryAlias a + : +- SubqueryAlias spark_catalog.default.tt4 + : +- Relation spark_catalog.default.tt4[f1#x] parquet + +- SubqueryAlias d + +- Project [f1#x] + +- Filter isnull(cast(udf(cast(f1#x as string)) as int)) + +- Join LeftOuter, (cast(udf(cast(f1#x as string)) as int) = cast(udf(cast(f1#x as string)) as int)) + :- SubqueryAlias b + : +- SubqueryAlias spark_catalog.default.tt3 + : +- Relation spark_catalog.default.tt3[f1#x,f2#x] parquet + +- SubqueryAlias c + +- SubqueryAlias spark_catalog.default.tt3 + +- Relation spark_catalog.default.tt3[f1#x,f2#x] parquet + + +-- !query +create or replace temporary view tt5 as select * from + (values (1, 10), (1, 11)) + as v(f1, f2) +-- !query analysis +CreateViewCommand `tt5`, select * from + (values (1, 10), (1, 11)) + as v(f1, f2), false, true, LocalTempView, true + +- Project [f1#x, f2#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x, col2#x AS f2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +create or replace temporary view tt6 as select * from + (values (1, 9), (1, 2), (2, 9)) + as v(f1, f2) +-- !query analysis +CreateViewCommand `tt6`, select * from + (values (1, 9), (1, 2), (2, 9)) + as v(f1, f2), false, true, LocalTempView, true + +- Project [f1#x, f2#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x, col2#x AS f2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from tt5,tt6 where udf(tt5.f1) = udf(tt6.f1) and udf(tt5.f1) = udf(udf(tt5.f2) - udf(tt6.f2)) +-- !query analysis +Project [f1#x, f2#x, f1#x, f2#x] ++- Filter ((cast(udf(cast(f1#x as string)) as int) = cast(udf(cast(f1#x as string)) as int)) AND (cast(udf(cast(f1#x as string)) as int) = cast(udf(cast((cast(udf(cast(f2#x as string)) as int) - cast(udf(cast(f2#x as string)) as int)) as string)) as int))) + +- Join Inner + :- SubqueryAlias tt5 + : +- View (`tt5`, [f1#x,f2#x]) + : +- Project [cast(f1#x as int) AS f1#x, cast(f2#x as int) AS f2#x] + : +- Project [f1#x, f2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x, col2#x AS f2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias tt6 + +- View (`tt6`, [f1#x,f2#x]) + +- Project [cast(f1#x as int) AS f1#x, cast(f2#x as int) AS f2#x] + +- Project [f1#x, f2#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x, col2#x AS f2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +create or replace temporary view xx as select * from + (values (1), (2), (3)) + as v(pkxx) +-- !query analysis +CreateViewCommand `xx`, select * from + (values (1), (2), (3)) + as v(pkxx), false, true, LocalTempView, true + +- Project [pkxx#x] + +- SubqueryAlias v + +- Project [col1#x AS pkxx#x] + +- LocalRelation [col1#x] + + +-- !query +create or replace temporary view yy as select * from + (values (101, 1), (201, 2), (301, NULL)) + as v(pkyy, pkxx) +-- !query analysis +CreateViewCommand `yy`, select * from + (values (101, 1), (201, 2), (301, NULL)) + as v(pkyy, pkxx), false, true, LocalTempView, true + +- Project [pkyy#x, pkxx#x] + +- SubqueryAlias v + +- Project [col1#x AS pkyy#x, col2#x AS pkxx#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select udf(udf(yy.pkyy)) as yy_pkyy, udf(yy.pkxx) as yy_pkxx, udf(yya.pkyy) as yya_pkyy, + udf(xxa.pkxx) as xxa_pkxx, udf(xxb.pkxx) as xxb_pkxx +from yy + left join (SELECT * FROM yy where pkyy = 101) as yya ON udf(yy.pkyy) = udf(yya.pkyy) + left join xx xxa on udf(yya.pkxx) = udf(udf(xxa.pkxx)) + left join xx xxb on udf(udf(coalesce (xxa.pkxx, 1))) = udf(xxb.pkxx) +-- !query analysis +Project [cast(udf(cast(cast(udf(cast(pkyy#x as string)) as int) as string)) as int) AS yy_pkyy#x, cast(udf(cast(pkxx#x as string)) as int) AS yy_pkxx#x, cast(udf(cast(pkyy#x as string)) as int) AS yya_pkyy#x, cast(udf(cast(pkxx#x as string)) as int) AS xxa_pkxx#x, cast(udf(cast(pkxx#x as string)) as int) AS xxb_pkxx#x] ++- Join LeftOuter, (cast(udf(cast(cast(udf(cast(coalesce(pkxx#x, 1) as string)) as int) as string)) as int) = cast(udf(cast(pkxx#x as string)) as int)) + :- Join LeftOuter, (cast(udf(cast(pkxx#x as string)) as int) = cast(udf(cast(cast(udf(cast(pkxx#x as string)) as int) as string)) as int)) + : :- Join LeftOuter, (cast(udf(cast(pkyy#x as string)) as int) = cast(udf(cast(pkyy#x as string)) as int)) + : : :- SubqueryAlias yy + : : : +- View (`yy`, [pkyy#x,pkxx#x]) + : : : +- Project [cast(pkyy#x as int) AS pkyy#x, cast(pkxx#x as int) AS pkxx#x] + : : : +- Project [pkyy#x, pkxx#x] + : : : +- SubqueryAlias v + : : : +- Project [col1#x AS pkyy#x, col2#x AS pkxx#x] + : : : +- LocalRelation [col1#x, col2#x] + : : +- SubqueryAlias yya + : : +- Project [pkyy#x, pkxx#x] + : : +- Filter (pkyy#x = 101) + : : +- SubqueryAlias yy + : : +- View (`yy`, [pkyy#x,pkxx#x]) + : : +- Project [cast(pkyy#x as int) AS pkyy#x, cast(pkxx#x as int) AS pkxx#x] + : : +- Project [pkyy#x, pkxx#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS pkyy#x, col2#x AS pkxx#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias xxa + : +- SubqueryAlias xx + : +- View (`xx`, [pkxx#x]) + : +- Project [cast(pkxx#x as int) AS pkxx#x] + : +- Project [pkxx#x] + : +- SubqueryAlias v + : +- Project [col1#x AS pkxx#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias xxb + +- SubqueryAlias xx + +- View (`xx`, [pkxx#x]) + +- Project [cast(pkxx#x as int) AS pkxx#x] + +- Project [pkxx#x] + +- SubqueryAlias v + +- Project [col1#x AS pkxx#x] + +- LocalRelation [col1#x] + + +-- !query +create or replace temporary view zt1 as select * from + (values (53)) + as v(f1) +-- !query analysis +CreateViewCommand `zt1`, select * from + (values (53)) + as v(f1), false, true, LocalTempView, true + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +create or replace temporary view zt2 as select * from + (values (53)) + as v(f2) +-- !query analysis +CreateViewCommand `zt2`, select * from + (values (53)) + as v(f2), false, true, LocalTempView, true + +- Project [f2#x] + +- SubqueryAlias v + +- Project [col1#x AS f2#x] + +- LocalRelation [col1#x] + + +-- !query +create or replace temporary view zt3(f3 int) using parquet +-- !query analysis +CreateTempViewUsing [tableIdent:`zt3` StructType(StructField(f3,IntegerType,true)) replace:true provider:parquet Map() + + +-- !query +select * from + zt2 left join zt3 on (udf(f2) = udf(udf(f3))) + left join zt1 on (udf(udf(f3)) = udf(f1)) +where udf(f2) = 53 +-- !query analysis +Project [f2#x, f3#x, f1#x] ++- Filter (cast(udf(cast(f2#x as string)) as int) = 53) + +- Join LeftOuter, (cast(udf(cast(cast(udf(cast(f3#x as string)) as int) as string)) as int) = cast(udf(cast(f1#x as string)) as int)) + :- Join LeftOuter, (cast(udf(cast(f2#x as string)) as int) = cast(udf(cast(cast(udf(cast(f3#x as string)) as int) as string)) as int)) + : :- SubqueryAlias zt2 + : : +- View (`zt2`, [f2#x]) + : : +- Project [cast(f2#x as int) AS f2#x] + : : +- Project [f2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS f2#x] + : : +- LocalRelation [col1#x] + : +- SubqueryAlias zt3 + : +- View (`zt3`, [f3#x]) + : +- Relation [f3#x] parquet + +- SubqueryAlias zt1 + +- View (`zt1`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +create temp view zv1 as select *,'dummy' AS junk from zt1 +-- !query analysis +CreateViewCommand `zv1`, select *,'dummy' AS junk from zt1, false, false, LocalTempView, true + +- Project [f1#x, dummy AS junk#x] + +- SubqueryAlias zt1 + +- View (`zt1`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from + zt2 left join zt3 on (f2 = udf(f3)) + left join zv1 on (udf(f3) = f1) +where udf(udf(f2)) = 53 +-- !query analysis +Project [f2#x, f3#x, f1#x, junk#x] ++- Filter (cast(udf(cast(cast(udf(cast(f2#x as string)) as int) as string)) as int) = 53) + +- Join LeftOuter, (cast(udf(cast(f3#x as string)) as int) = f1#x) + :- Join LeftOuter, (f2#x = cast(udf(cast(f3#x as string)) as int)) + : :- SubqueryAlias zt2 + : : +- View (`zt2`, [f2#x]) + : : +- Project [cast(f2#x as int) AS f2#x] + : : +- Project [f2#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS f2#x] + : : +- LocalRelation [col1#x] + : +- SubqueryAlias zt3 + : +- View (`zt3`, [f3#x]) + : +- Relation [f3#x] parquet + +- SubqueryAlias zv1 + +- View (`zv1`, [f1#x,junk#x]) + +- Project [cast(f1#x as int) AS f1#x, cast(junk#x as string) AS junk#x] + +- Project [f1#x, dummy AS junk#x] + +- SubqueryAlias zt1 + +- View (`zt1`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select udf(a.unique2), udf(a.ten), udf(b.tenthous), udf(b.unique2), udf(b.hundred) +from tenk1 a left join tenk1 b on a.unique2 = udf(b.tenthous) +where udf(a.unique1) = 42 and + ((udf(b.unique2) is null and udf(a.ten) = 2) or udf(udf(b.hundred)) = udf(udf(3))) +-- !query analysis +Project [cast(udf(cast(unique2#x as string)) as int) AS udf(unique2)#x, cast(udf(cast(ten#x as string)) as int) AS udf(ten)#x, cast(udf(cast(tenthous#x as string)) as int) AS udf(tenthous)#x, cast(udf(cast(unique2#x as string)) as int) AS udf(unique2)#x, cast(udf(cast(hundred#x as string)) as int) AS udf(hundred)#x] ++- Filter ((cast(udf(cast(unique1#x as string)) as int) = 42) AND ((isnull(cast(udf(cast(unique2#x as string)) as int)) AND (cast(udf(cast(ten#x as string)) as int) = 2)) OR (cast(udf(cast(cast(udf(cast(hundred#x as string)) as int) as string)) as int) = cast(udf(cast(cast(udf(cast(3 as string)) as int) as string)) as int)))) + +- Join LeftOuter, (unique2#x = cast(udf(cast(tenthous#x as string)) as int)) + :- SubqueryAlias a + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +create or replace temporary view a (i integer) using parquet +-- !query analysis +CreateTempViewUsing [tableIdent:`a` StructType(StructField(i,IntegerType,true)) replace:true provider:parquet Map() + + +-- !query +create or replace temporary view b (x integer, y integer) using parquet +-- !query analysis +CreateTempViewUsing [tableIdent:`b` StructType(StructField(x,IntegerType,true),StructField(y,IntegerType,true)) replace:true provider:parquet Map() + + +-- !query +select * from a left join b on udf(i) = x and i = udf(y) and udf(x) = udf(i) +-- !query analysis +Project [i#x, x#x, y#x] ++- Join LeftOuter, (((cast(udf(cast(i#x as string)) as int) = x#x) AND (i#x = cast(udf(cast(y#x as string)) as int))) AND (cast(udf(cast(x#x as string)) as int) = cast(udf(cast(i#x as string)) as int))) + :- SubqueryAlias a + : +- View (`a`, [i#x]) + : +- Relation [i#x] parquet + +- SubqueryAlias b + +- View (`b`, [x#x,y#x]) + +- Relation [x#x,y#x] parquet + + +-- !query +select udf(t1.q2), udf(count(t2.q1, t2.q2)) +from int8_tbl t1 left join int8_tbl t2 on (udf(udf(t1.q2)) = t2.q1) +group by udf(t1.q2) order by 1 +-- !query analysis +Sort [udf(q2)#xL ASC NULLS FIRST], true ++- Aggregate [cast(udf(cast(q2#xL as string)) as bigint)], [cast(udf(cast(q2#xL as string)) as bigint) AS udf(q2)#xL, cast(udf(cast(count(q1#xL, q2#xL) as string)) as bigint) AS udf(count(q1, q2))#xL] + +- Join LeftOuter, (cast(udf(cast(cast(udf(cast(q2#xL as string)) as bigint) as string)) as bigint) = q1#xL) + :- SubqueryAlias t1 + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias t2 + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select udf(udf(t1.q2)), udf(count(t2.q1, t2.q2)) +from int8_tbl t1 left join (select * from int8_tbl) t2 on (udf(udf(t1.q2)) = udf(t2.q1)) +group by udf(udf(t1.q2)) order by 1 +-- !query analysis +Sort [udf(udf(q2))#xL ASC NULLS FIRST], true ++- Aggregate [cast(udf(cast(cast(udf(cast(q2#xL as string)) as bigint) as string)) as bigint)], [cast(udf(cast(cast(udf(cast(q2#xL as string)) as bigint) as string)) as bigint) AS udf(udf(q2))#xL, cast(udf(cast(count(q1#xL, q2#xL) as string)) as bigint) AS udf(count(q1, q2))#xL] + +- Join LeftOuter, (cast(udf(cast(cast(udf(cast(q2#xL as string)) as bigint) as string)) as bigint) = cast(udf(cast(q1#xL as string)) as bigint)) + :- SubqueryAlias t1 + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias t2 + +- Project [q1#xL, q2#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select udf(t1.q2) as q2, udf(udf(count(t2.q1, t2.q2))) +from int8_tbl t1 left join + (select udf(q1) as q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2 + on (udf(t1.q2) = udf(t2.q1)) +group by t1.q2 order by 1 +-- !query analysis +Sort [q2#xL ASC NULLS FIRST], true ++- Aggregate [q2#xL], [cast(udf(cast(q2#xL as string)) as bigint) AS q2#xL, cast(udf(cast(cast(udf(cast(count(q1#xL, q2#xL) as string)) as bigint) as string)) as bigint) AS udf(udf(count(q1, q2)))#xL] + +- Join LeftOuter, (cast(udf(cast(q2#xL as string)) as bigint) = cast(udf(cast(q1#xL as string)) as bigint)) + :- SubqueryAlias t1 + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias t2 + +- Project [cast(udf(cast(q1#xL as string)) as bigint) AS q1#xL, CASE WHEN (q2#xL = cast(1 as bigint)) THEN cast(1 as bigint) ELSE q2#xL END AS q2#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +create or replace temporary view a as select * from + (values ('p'), ('q')) + as v(code) +-- !query analysis +CreateViewCommand `a`, select * from + (values ('p'), ('q')) + as v(code), false, true, LocalTempView, true + +- Project [code#x] + +- SubqueryAlias v + +- Project [col1#x AS code#x] + +- LocalRelation [col1#x] + + +-- !query +create or replace temporary view b as select * from + (values ('p', 1), ('p', 2)) + as v(a, num) +-- !query analysis +CreateViewCommand `b`, select * from + (values ('p', 1), ('p', 2)) + as v(a, num), false, true, LocalTempView, true + +- Project [a#x, num#x] + +- SubqueryAlias v + +- Project [col1#x AS a#x, col2#x AS num#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +create or replace temporary view c as select * from + (values ('A', 'p'), ('B', 'q'), ('C', null)) + as v(name, a) +-- !query analysis +CreateViewCommand `c`, select * from + (values ('A', 'p'), ('B', 'q'), ('C', null)) + as v(name, a), false, true, LocalTempView, true + +- Project [name#x, a#x] + +- SubqueryAlias v + +- Project [col1#x AS name#x, col2#x AS a#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select udf(c.name), udf(ss.code), udf(ss.b_cnt), udf(ss.const) +from c left join + (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const + from a left join + (select udf(count(1)) as cnt, b.a as a from b group by b.a) as b_grp + on udf(a.code) = udf(udf(b_grp.a)) + ) as ss + on (udf(udf(c.a)) = udf(ss.code)) +order by c.name +-- !query analysis +Project [udf(name)#x, udf(code)#x, udf(b_cnt)#xL, udf(const)#x] ++- Sort [name#x ASC NULLS FIRST], true + +- Project [cast(udf(cast(name#x as string)) as string) AS udf(name)#x, cast(udf(cast(code#x as string)) as string) AS udf(code)#x, cast(udf(cast(b_cnt#xL as string)) as bigint) AS udf(b_cnt)#xL, cast(udf(cast(const#x as string)) as int) AS udf(const)#x, name#x] + +- Join LeftOuter, (cast(udf(cast(cast(udf(cast(a#x as string)) as string) as string)) as string) = cast(udf(cast(code#x as string)) as string)) + :- SubqueryAlias c + : +- View (`c`, [name#x,a#x]) + : +- Project [cast(name#x as string) AS name#x, cast(a#x as string) AS a#x] + : +- Project [name#x, a#x] + : +- SubqueryAlias v + : +- Project [col1#x AS name#x, col2#x AS a#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias ss + +- Project [code#x, coalesce(cnt#xL, cast(0 as bigint)) AS b_cnt#xL, -1 AS const#x] + +- Join LeftOuter, (cast(udf(cast(code#x as string)) as string) = cast(udf(cast(cast(udf(cast(a#x as string)) as string) as string)) as string)) + :- SubqueryAlias a + : +- View (`a`, [code#x]) + : +- Project [cast(code#x as string) AS code#x] + : +- Project [code#x] + : +- SubqueryAlias v + : +- Project [col1#x AS code#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias b_grp + +- Aggregate [a#x], [cast(udf(cast(count(1) as string)) as bigint) AS cnt#xL, a#x AS a#x] + +- SubqueryAlias b + +- View (`b`, [a#x,num#x]) + +- Project [cast(a#x as string) AS a#x, cast(num#x as int) AS num#x] + +- Project [a#x, num#x] + +- SubqueryAlias v + +- Project [col1#x AS a#x, col2#x AS num#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT sub3.key3, sub4.value2, COALESCE(sub4.value2, 66) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT udf(sub5.key5) as key5, udf(udf(COALESCE(sub6.value1, 1))) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON sub5.key5 = udf(sub6.key6) + ) sub4 + ON udf(sub4.key5) = sub3.key3 +) sub2 +ON udf(udf(sub1.key1)) = udf(udf(sub2.key3)) +-- !query analysis +Project [key1#x, key3#x, value2#x, value3#x] ++- Join LeftOuter, (cast(udf(cast(cast(udf(cast(key1#x as string)) as int) as string)) as int) = cast(udf(cast(cast(udf(cast(key3#x as string)) as int) as string)) as int)) + :- SubqueryAlias sub1 + : +- Project [1 AS key1#x] + : +- OneRowRelation + +- SubqueryAlias sub2 + +- Project [key3#x, value2#x, coalesce(value2#x, 66) AS value3#x] + +- Join LeftOuter, (cast(udf(cast(key5#x as string)) as int) = key3#x) + :- SubqueryAlias sub3 + : +- Project [1 AS key3#x] + : +- OneRowRelation + +- SubqueryAlias sub4 + +- Project [cast(udf(cast(key5#x as string)) as int) AS key5#x, cast(udf(cast(cast(udf(cast(coalesce(value1#x, 1) as string)) as int) as string)) as int) AS value2#x] + +- Join LeftOuter, (key5#x = cast(udf(cast(key6#x as string)) as int)) + :- SubqueryAlias sub5 + : +- Project [1 AS key5#x] + : +- OneRowRelation + +- SubqueryAlias sub6 + +- Project [2 AS key6#x, 42 AS value1#x] + +- OneRowRelation + + +-- !query +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT udf(sub3.key3) as key3, udf(value2), udf(COALESCE(value2, 66)) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON udf(udf(sub5.key5)) = sub6.key6 + ) sub4 + ON sub4.key5 = sub3.key3 +) sub2 +ON sub1.key1 = udf(udf(sub2.key3)) +-- !query analysis +Project [key1#x, key3#x, udf(value2)#x, value3#x] ++- Join LeftOuter, (key1#x = cast(udf(cast(cast(udf(cast(key3#x as string)) as int) as string)) as int)) + :- SubqueryAlias sub1 + : +- Project [1 AS key1#x] + : +- OneRowRelation + +- SubqueryAlias sub2 + +- Project [cast(udf(cast(key3#x as string)) as int) AS key3#x, cast(udf(cast(value2#x as string)) as int) AS udf(value2)#x, cast(udf(cast(coalesce(value2#x, 66) as string)) as int) AS value3#x] + +- Join LeftOuter, (key5#x = key3#x) + :- SubqueryAlias sub3 + : +- Project [1 AS key3#x] + : +- OneRowRelation + +- SubqueryAlias sub4 + +- Project [key5#x, coalesce(value1#x, 1) AS value2#x] + +- Join LeftOuter, (cast(udf(cast(cast(udf(cast(key5#x as string)) as int) as string)) as int) = key6#x) + :- SubqueryAlias sub5 + : +- Project [1 AS key5#x] + : +- OneRowRelation + +- SubqueryAlias sub6 + +- Project [2 AS key6#x, 42 AS value1#x] + +- OneRowRelation + + +-- !query +SELECT udf(qq), udf(udf(unique1)) + FROM + ( SELECT udf(COALESCE(q1, 0)) AS qq FROM int8_tbl a ) AS ss1 + FULL OUTER JOIN + ( SELECT udf(udf(COALESCE(q2, -1))) AS qq FROM int8_tbl b ) AS ss2 + USING (qq) + INNER JOIN tenk1 c ON udf(qq) = udf(unique2) +-- !query analysis +Project [cast(udf(cast(qq#xL as string)) as bigint) AS udf(qq)#xL, cast(udf(cast(cast(udf(cast(unique1#x as string)) as int) as string)) as int) AS udf(udf(unique1))#x] ++- Join Inner, (cast(udf(cast(qq#xL as string)) as bigint) = cast(cast(udf(cast(unique2#x as string)) as int) as bigint)) + :- Project [coalesce(qq#xL, qq#xL) AS qq#xL] + : +- Join FullOuter, (qq#xL = qq#xL) + : :- SubqueryAlias ss1 + : : +- Project [cast(udf(cast(coalesce(q1#xL, cast(0 as bigint)) as string)) as bigint) AS qq#xL] + : : +- SubqueryAlias a + : : +- SubqueryAlias int8_tbl + : : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : : +- Project [q1#xL, q2#xL] + : : +- SubqueryAlias v + : : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : : +- LocalRelation [col1#xL, col2#xL] + : +- SubqueryAlias ss2 + : +- Project [cast(udf(cast(cast(udf(cast(coalesce(q2#xL, cast(-1 as bigint)) as string)) as bigint) as string)) as bigint) AS qq#xL] + : +- SubqueryAlias b + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias c + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +create or replace temporary view nt1 as select * from + (values(1,true,true), (2,true,false), (3,false,false)) + as v(id, a1, a2) +-- !query analysis +CreateViewCommand `nt1`, select * from + (values(1,true,true), (2,true,false), (3,false,false)) + as v(id, a1, a2), false, true, LocalTempView, true + +- Project [id#x, a1#x, a2#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x, col2#x AS a1#x, col3#x AS a2#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +create or replace temporary view nt2 as select * from + (values(1,1,true,true), (2,2,true,false), (3,3,false,false)) + as v(id, nt1_id, b1, b2) +-- !query analysis +CreateViewCommand `nt2`, select * from + (values(1,1,true,true), (2,2,true,false), (3,3,false,false)) + as v(id, nt1_id, b1, b2), false, true, LocalTempView, true + +- Project [id#x, nt1_id#x, b1#x, b2#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x, col2#x AS nt1_id#x, col3#x AS b1#x, col4#x AS b2#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +create or replace temporary view nt3 as select * from + (values(1,1,true), (2,2,false), (3,3,true)) + as v(id, nt2_id, c1) +-- !query analysis +CreateViewCommand `nt3`, select * from + (values(1,1,true), (2,2,false), (3,3,true)) + as v(id, nt2_id, c1), false, true, LocalTempView, true + +- Project [id#x, nt2_id#x, c1#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x, col2#x AS nt2_id#x, col3#x AS c1#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +select udf(nt3.id) +from nt3 as nt3 + left join + (select nt2.*, (udf(nt2.b1) and udf(ss1.a3)) AS b3 + from nt2 as nt2 + left join + (select nt1.*, (udf(nt1.id) is not null) as a3 from nt1) as ss1 + on ss1.id = udf(udf(nt2.nt1_id)) + ) as ss2 + on udf(ss2.id) = nt3.nt2_id +where udf(nt3.id) = 1 and udf(ss2.b3) +-- !query analysis +Project [cast(udf(cast(id#x as string)) as int) AS udf(id)#x] ++- Filter ((cast(udf(cast(id#x as string)) as int) = 1) AND cast(udf(cast(b3#x as string)) as boolean)) + +- Join LeftOuter, (cast(udf(cast(id#x as string)) as int) = nt2_id#x) + :- SubqueryAlias nt3 + : +- SubqueryAlias nt3 + : +- View (`nt3`, [id#x,nt2_id#x,c1#x]) + : +- Project [cast(id#x as int) AS id#x, cast(nt2_id#x as int) AS nt2_id#x, cast(c1#x as boolean) AS c1#x] + : +- Project [id#x, nt2_id#x, c1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS id#x, col2#x AS nt2_id#x, col3#x AS c1#x] + : +- LocalRelation [col1#x, col2#x, col3#x] + +- SubqueryAlias ss2 + +- Project [id#x, nt1_id#x, b1#x, b2#x, (cast(udf(cast(b1#x as string)) as boolean) AND cast(udf(cast(a3#x as string)) as boolean)) AS b3#x] + +- Join LeftOuter, (id#x = cast(udf(cast(cast(udf(cast(nt1_id#x as string)) as int) as string)) as int)) + :- SubqueryAlias nt2 + : +- SubqueryAlias nt2 + : +- View (`nt2`, [id#x,nt1_id#x,b1#x,b2#x]) + : +- Project [cast(id#x as int) AS id#x, cast(nt1_id#x as int) AS nt1_id#x, cast(b1#x as boolean) AS b1#x, cast(b2#x as boolean) AS b2#x] + : +- Project [id#x, nt1_id#x, b1#x, b2#x] + : +- SubqueryAlias v + : +- Project [col1#x AS id#x, col2#x AS nt1_id#x, col3#x AS b1#x, col4#x AS b2#x] + : +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + +- SubqueryAlias ss1 + +- Project [id#x, a1#x, a2#x, isnotnull(cast(udf(cast(id#x as string)) as int)) AS a3#x] + +- SubqueryAlias nt1 + +- View (`nt1`, [id#x,a1#x,a2#x]) + +- Project [cast(id#x as int) AS id#x, cast(a1#x as boolean) AS a1#x, cast(a2#x as boolean) AS a2#x] + +- Project [id#x, a1#x, a2#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x, col2#x AS a1#x, col3#x AS a2#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +select * from int4_tbl a full join int4_tbl b on true +-- !query analysis +Project [f1#x, f1#x] ++- Join FullOuter, true + :- SubqueryAlias a + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias b + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from int4_tbl a full join int4_tbl b on false +-- !query analysis +Project [f1#x, f1#x] ++- Join FullOuter, false + :- SubqueryAlias a + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias b + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select udf(count(*)) from + tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2) + left join tenk1 c on udf(a.unique2) = udf(b.unique1) and udf(c.thousand) = udf(udf(a.thousand)) + join int4_tbl on udf(b.thousand) = f1 +-- !query analysis +Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] ++- Join Inner, (cast(udf(cast(thousand#x as string)) as int) = f1#x) + :- Join LeftOuter, ((cast(udf(cast(unique2#x as string)) as int) = cast(udf(cast(unique1#x as string)) as int)) AND (cast(udf(cast(thousand#x as string)) as int) = cast(udf(cast(cast(udf(cast(thousand#x as string)) as int) as string)) as int))) + : :- Join Inner, (cast(udf(cast(unique1#x as string)) as int) = cast(udf(cast(unique2#x as string)) as int)) + : : :- SubqueryAlias a + : : : +- SubqueryAlias spark_catalog.default.tenk1 + : : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : : +- SubqueryAlias b + : : +- SubqueryAlias spark_catalog.default.tenk1 + : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : +- SubqueryAlias c + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select udf(b.unique1) from + tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2) + left join tenk1 c on udf(b.unique1) = 42 and c.thousand = udf(a.thousand) + join int4_tbl i1 on udf(b.thousand) = udf(udf(f1)) + right join int4_tbl i2 on udf(udf(i2.f1)) = udf(b.tenthous) + order by udf(1) +-- !query analysis +Sort [cast(udf(cast(1 as string)) as int) ASC NULLS FIRST], true ++- Project [cast(udf(cast(unique1#x as string)) as int) AS udf(unique1)#x] + +- Join RightOuter, (cast(udf(cast(cast(udf(cast(f1#x as string)) as int) as string)) as int) = cast(udf(cast(tenthous#x as string)) as int)) + :- Join Inner, (cast(udf(cast(thousand#x as string)) as int) = cast(udf(cast(cast(udf(cast(f1#x as string)) as int) as string)) as int)) + : :- Join LeftOuter, ((cast(udf(cast(unique1#x as string)) as int) = 42) AND (thousand#x = cast(udf(cast(thousand#x as string)) as int))) + : : :- Join Inner, (cast(udf(cast(unique1#x as string)) as int) = cast(udf(cast(unique2#x as string)) as int)) + : : : :- SubqueryAlias a + : : : : +- SubqueryAlias spark_catalog.default.tenk1 + : : : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : : : +- SubqueryAlias b + : : : +- SubqueryAlias spark_catalog.default.tenk1 + : : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : : +- SubqueryAlias c + : : +- SubqueryAlias spark_catalog.default.tenk1 + : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : +- SubqueryAlias i1 + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias i2 + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from +( + select udf(unique1), udf(q1), udf(udf(coalesce(unique1, -1)) + udf(q1)) as fault + from int8_tbl left join tenk1 on (udf(q2) = udf(unique2)) +) ss +where udf(fault) = udf(122) +order by udf(fault) +-- !query analysis +Sort [cast(udf(cast(fault#xL as string)) as bigint) ASC NULLS FIRST], true ++- Project [udf(unique1)#x, udf(q1)#xL, fault#xL] + +- Filter (cast(udf(cast(fault#xL as string)) as bigint) = cast(cast(udf(cast(122 as string)) as int) as bigint)) + +- SubqueryAlias ss + +- Project [cast(udf(cast(unique1#x as string)) as int) AS udf(unique1)#x, cast(udf(cast(q1#xL as string)) as bigint) AS udf(q1)#xL, cast(udf(cast((cast(cast(udf(cast(coalesce(unique1#x, -1) as string)) as int) as bigint) + cast(udf(cast(q1#xL as string)) as bigint)) as string)) as bigint) AS fault#xL] + +- Join LeftOuter, (cast(udf(cast(q2#xL as string)) as bigint) = cast(cast(udf(cast(unique2#x as string)) as int) as bigint)) + :- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select udf(q1), udf(unique2), udf(thousand), udf(hundred) + from int8_tbl a left join tenk1 b on udf(q1) = udf(unique2) + where udf(coalesce(thousand,123)) = udf(q1) and udf(q1) = udf(udf(coalesce(hundred,123))) +-- !query analysis +Project [cast(udf(cast(q1#xL as string)) as bigint) AS udf(q1)#xL, cast(udf(cast(unique2#x as string)) as int) AS udf(unique2)#x, cast(udf(cast(thousand#x as string)) as int) AS udf(thousand)#x, cast(udf(cast(hundred#x as string)) as int) AS udf(hundred)#x] ++- Filter ((cast(cast(udf(cast(coalesce(thousand#x, 123) as string)) as int) as bigint) = cast(udf(cast(q1#xL as string)) as bigint)) AND (cast(udf(cast(q1#xL as string)) as bigint) = cast(cast(udf(cast(cast(udf(cast(coalesce(hundred#x, 123) as string)) as int) as string)) as int) as bigint))) + +- Join LeftOuter, (cast(udf(cast(q1#xL as string)) as bigint) = cast(cast(udf(cast(unique2#x as string)) as int) as bigint)) + :- SubqueryAlias a + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select udf(f1), udf(unique2), case when udf(udf(unique2)) is null then udf(f1) else 0 end + from int4_tbl a left join tenk1 b on udf(f1) = udf(udf(unique2)) + where (case when udf(unique2) is null then udf(f1) else 0 end) = 0 +-- !query analysis +Project [cast(udf(cast(f1#x as string)) as int) AS udf(f1)#x, cast(udf(cast(unique2#x as string)) as int) AS udf(unique2)#x, CASE WHEN isnull(cast(udf(cast(cast(udf(cast(unique2#x as string)) as int) as string)) as int)) THEN cast(udf(cast(f1#x as string)) as int) ELSE 0 END AS CASE WHEN (udf(udf(unique2)) IS NULL) THEN udf(f1) ELSE 0 END#x] ++- Filter (CASE WHEN isnull(cast(udf(cast(unique2#x as string)) as int)) THEN cast(udf(cast(f1#x as string)) as int) ELSE 0 END = 0) + +- Join LeftOuter, (cast(udf(cast(f1#x as string)) as int) = cast(udf(cast(cast(udf(cast(unique2#x as string)) as int) as string)) as int)) + :- SubqueryAlias a + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias b + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select udf(a.unique1), udf(b.unique1), udf(c.unique1), udf(coalesce(b.twothousand, a.twothousand)) + from tenk1 a left join tenk1 b on udf(b.thousand) = a.unique1 left join tenk1 c on udf(c.unique2) = udf(coalesce(b.twothousand, a.twothousand)) + where a.unique2 < udf(10) and udf(udf(coalesce(b.twothousand, a.twothousand))) = udf(44) +-- !query analysis +Project [cast(udf(cast(unique1#x as string)) as int) AS udf(unique1)#x, cast(udf(cast(unique1#x as string)) as int) AS udf(unique1)#x, cast(udf(cast(unique1#x as string)) as int) AS udf(unique1)#x, cast(udf(cast(coalesce(twothousand#x, twothousand#x) as string)) as int) AS udf(coalesce(twothousand, twothousand))#x] ++- Filter ((unique2#x < cast(udf(cast(10 as string)) as int)) AND (cast(udf(cast(cast(udf(cast(coalesce(twothousand#x, twothousand#x) as string)) as int) as string)) as int) = cast(udf(cast(44 as string)) as int))) + +- Join LeftOuter, (cast(udf(cast(unique2#x as string)) as int) = cast(udf(cast(coalesce(twothousand#x, twothousand#x) as string)) as int)) + :- Join LeftOuter, (cast(udf(cast(thousand#x as string)) as int) = unique1#x) + : :- SubqueryAlias a + : : +- SubqueryAlias spark_catalog.default.tenk1 + : : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + : +- SubqueryAlias b + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias c + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select * from + text_tbl t1 + inner join int8_tbl i8 + on udf(i8.q2) = udf(udf(456)) + right join text_tbl t2 + on udf(t1.f1) = udf(udf('doh!')) + left join int4_tbl i4 + on udf(udf(i8.q1)) = i4.f1 +-- !query analysis +Project [f1#x, q1#xL, q2#xL, f1#x, f1#x] ++- Join LeftOuter, (cast(udf(cast(cast(udf(cast(q1#xL as string)) as bigint) as string)) as bigint) = cast(f1#x as bigint)) + :- Join RightOuter, (cast(udf(cast(f1#x as string)) as string) = cast(udf(cast(cast(udf(cast(doh! as string)) as string) as string)) as string)) + : :- Join Inner, (cast(udf(cast(q2#xL as string)) as bigint) = cast(cast(udf(cast(cast(udf(cast(456 as string)) as int) as string)) as int) as bigint)) + : : :- SubqueryAlias t1 + : : : +- SubqueryAlias text_tbl + : : : +- View (`TEXT_TBL`, [f1#x]) + : : : +- Project [cast(f1#x as string) AS f1#x] + : : : +- Project [f1#x] + : : : +- SubqueryAlias v + : : : +- Project [col1#x AS f1#x] + : : : +- LocalRelation [col1#x] + : : +- SubqueryAlias i8 + : : +- SubqueryAlias int8_tbl + : : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : : +- Project [q1#xL, q2#xL] + : : +- SubqueryAlias v + : : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : : +- LocalRelation [col1#xL, col2#xL] + : +- SubqueryAlias t2 + : +- SubqueryAlias text_tbl + : +- View (`TEXT_TBL`, [f1#x]) + : +- Project [cast(f1#x as string) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias i4 + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from + (select udf(udf(1)) as id) as xx + left join + (tenk1 as a1 full join (select udf(1) as id) as yy on (udf(a1.unique1) = udf(yy.id))) + on (xx.id = udf(udf(coalesce(yy.id)))) +-- !query analysis +Project [id#x, unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, id#x] ++- Join LeftOuter, (id#x = cast(udf(cast(cast(udf(cast(coalesce(id#x) as string)) as int) as string)) as int)) + :- SubqueryAlias xx + : +- Project [cast(udf(cast(cast(udf(cast(1 as string)) as int) as string)) as int) AS id#x] + : +- OneRowRelation + +- Join FullOuter, (cast(udf(cast(unique1#x as string)) as int) = cast(udf(cast(id#x as string)) as int)) + :- SubqueryAlias a1 + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias yy + +- Project [cast(udf(cast(1 as string)) as int) AS id#x] + +- OneRowRelation + + +-- !query +select udf(a.q2), udf(b.q1) + from int8_tbl a left join int8_tbl b on udf(a.q2) = coalesce(b.q1, 1) + where udf(udf(coalesce(b.q1, 1)) > 0) +-- !query analysis +Project [cast(udf(cast(q2#xL as string)) as bigint) AS udf(q2)#xL, cast(udf(cast(q1#xL as string)) as bigint) AS udf(q1)#xL] ++- Filter cast(udf(cast((cast(udf(cast(coalesce(q1#xL, cast(1 as bigint)) as string)) as bigint) > cast(0 as bigint)) as string)) as boolean) + +- Join LeftOuter, (cast(udf(cast(q2#xL as string)) as bigint) = coalesce(q1#xL, cast(1 as bigint))) + :- SubqueryAlias a + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias b + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +create or replace temporary view parent as select * from + (values (1, 10), (2, 20), (3, 30)) + as v(k, pd) +-- !query analysis +CreateViewCommand `parent`, select * from + (values (1, 10), (2, 20), (3, 30)) + as v(k, pd), false, true, LocalTempView, true + +- Project [k#x, pd#x] + +- SubqueryAlias v + +- Project [col1#x AS k#x, col2#x AS pd#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +create or replace temporary view child as select * from + (values (1, 100), (4, 400)) + as v(k, cd) +-- !query analysis +CreateViewCommand `child`, select * from + (values (1, 100), (4, 400)) + as v(k, cd), false, true, LocalTempView, true + +- Project [k#x, cd#x] + +- SubqueryAlias v + +- Project [col1#x AS k#x, col2#x AS cd#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select p.* from parent p left join child c on (udf(p.k) = udf(c.k)) +-- !query analysis +Project [k#x, pd#x] ++- Join LeftOuter, (cast(udf(cast(k#x as string)) as int) = cast(udf(cast(k#x as string)) as int)) + :- SubqueryAlias p + : +- SubqueryAlias parent + : +- View (`parent`, [k#x,pd#x]) + : +- Project [cast(k#x as int) AS k#x, cast(pd#x as int) AS pd#x] + : +- Project [k#x, pd#x] + : +- SubqueryAlias v + : +- Project [col1#x AS k#x, col2#x AS pd#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias c + +- SubqueryAlias child + +- View (`child`, [k#x,cd#x]) + +- Project [cast(k#x as int) AS k#x, cast(cd#x as int) AS cd#x] + +- Project [k#x, cd#x] + +- SubqueryAlias v + +- Project [col1#x AS k#x, col2#x AS cd#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select p.*, linked from parent p + left join (select c.*, udf(udf(true)) as linked from child c) as ss + on (udf(p.k) = udf(udf(ss.k))) +-- !query analysis +Project [k#x, pd#x, linked#x] ++- Join LeftOuter, (cast(udf(cast(k#x as string)) as int) = cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int)) + :- SubqueryAlias p + : +- SubqueryAlias parent + : +- View (`parent`, [k#x,pd#x]) + : +- Project [cast(k#x as int) AS k#x, cast(pd#x as int) AS pd#x] + : +- Project [k#x, pd#x] + : +- SubqueryAlias v + : +- Project [col1#x AS k#x, col2#x AS pd#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias ss + +- Project [k#x, cd#x, cast(udf(cast(cast(udf(cast(true as string)) as boolean) as string)) as boolean) AS linked#x] + +- SubqueryAlias c + +- SubqueryAlias child + +- View (`child`, [k#x,cd#x]) + +- Project [cast(k#x as int) AS k#x, cast(cd#x as int) AS cd#x] + +- Project [k#x, cd#x] + +- SubqueryAlias v + +- Project [col1#x AS k#x, col2#x AS cd#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select p.* from + parent p left join child c on (udf(p.k) = c.k) + where p.k = udf(1) and udf(udf(p.k)) = udf(udf(2)) +-- !query analysis +Project [k#x, pd#x] ++- Filter ((k#x = cast(udf(cast(1 as string)) as int)) AND (cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int) = cast(udf(cast(cast(udf(cast(2 as string)) as int) as string)) as int))) + +- Join LeftOuter, (cast(udf(cast(k#x as string)) as int) = k#x) + :- SubqueryAlias p + : +- SubqueryAlias parent + : +- View (`parent`, [k#x,pd#x]) + : +- Project [cast(k#x as int) AS k#x, cast(pd#x as int) AS pd#x] + : +- Project [k#x, pd#x] + : +- SubqueryAlias v + : +- Project [col1#x AS k#x, col2#x AS pd#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias c + +- SubqueryAlias child + +- View (`child`, [k#x,cd#x]) + +- Project [cast(k#x as int) AS k#x, cast(cd#x as int) AS cd#x] + +- Project [k#x, cd#x] + +- SubqueryAlias v + +- Project [col1#x AS k#x, col2#x AS cd#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select p.* from + (parent p left join child c on (udf(p.k) = c.k)) join parent x on p.k = udf(x.k) + where udf(p.k) = udf(1) and udf(udf(p.k)) = udf(udf(2)) +-- !query analysis +Project [k#x, pd#x] ++- Filter ((cast(udf(cast(k#x as string)) as int) = cast(udf(cast(1 as string)) as int)) AND (cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int) = cast(udf(cast(cast(udf(cast(2 as string)) as int) as string)) as int))) + +- Join Inner, (k#x = cast(udf(cast(k#x as string)) as int)) + :- Join LeftOuter, (cast(udf(cast(k#x as string)) as int) = k#x) + : :- SubqueryAlias p + : : +- SubqueryAlias parent + : : +- View (`parent`, [k#x,pd#x]) + : : +- Project [cast(k#x as int) AS k#x, cast(pd#x as int) AS pd#x] + : : +- Project [k#x, pd#x] + : : +- SubqueryAlias v + : : +- Project [col1#x AS k#x, col2#x AS pd#x] + : : +- LocalRelation [col1#x, col2#x] + : +- SubqueryAlias c + : +- SubqueryAlias child + : +- View (`child`, [k#x,cd#x]) + : +- Project [cast(k#x as int) AS k#x, cast(cd#x as int) AS cd#x] + : +- Project [k#x, cd#x] + : +- SubqueryAlias v + : +- Project [col1#x AS k#x, col2#x AS cd#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias x + +- SubqueryAlias parent + +- View (`parent`, [k#x,pd#x]) + +- Project [cast(k#x as int) AS k#x, cast(pd#x as int) AS pd#x] + +- Project [k#x, pd#x] + +- SubqueryAlias v + +- Project [col1#x AS k#x, col2#x AS pd#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +create or replace temporary view a as select * from + (values (0), (1)) + as v(id) +-- !query analysis +CreateViewCommand `a`, select * from + (values (0), (1)) + as v(id), false, true, LocalTempView, true + +- Project [id#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x] + +- LocalRelation [col1#x] + + +-- !query +create or replace temporary view b as select * from + (values (0, 0), (1, NULL)) + as v(id, a_id) +-- !query analysis +CreateViewCommand `b`, select * from + (values (0, 0), (1, NULL)) + as v(id, a_id), false, true, LocalTempView, true + +- Project [id#x, a_id#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x, col2#x AS a_id#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(udf(a.id)) IS NULL OR udf(a.id) > 0) +-- !query analysis +Project [id#x, a_id#x, id#x] ++- Filter (isnull(cast(udf(cast(cast(udf(cast(id#x as string)) as int) as string)) as int)) OR (cast(udf(cast(id#x as string)) as int) > 0)) + +- Join LeftOuter, (cast(udf(cast(a_id#x as string)) as int) = cast(udf(cast(id#x as string)) as int)) + :- SubqueryAlias b + : +- View (`b`, [id#x,a_id#x]) + : +- Project [cast(id#x as int) AS id#x, cast(a_id#x as int) AS a_id#x] + : +- Project [id#x, a_id#x] + : +- SubqueryAlias v + : +- Project [col1#x AS id#x, col2#x AS a_id#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias a + +- View (`a`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT b.* FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(a.id) IS NULL OR udf(udf(a.id)) > 0) +-- !query analysis +Project [id#x, a_id#x] ++- Filter (isnull(cast(udf(cast(id#x as string)) as int)) OR (cast(udf(cast(cast(udf(cast(id#x as string)) as int) as string)) as int) > 0)) + +- Join LeftOuter, (cast(udf(cast(a_id#x as string)) as int) = cast(udf(cast(id#x as string)) as int)) + :- SubqueryAlias b + : +- View (`b`, [id#x,a_id#x]) + : +- Project [cast(id#x as int) AS id#x, cast(a_id#x as int) AS a_id#x] + : +- Project [id#x, a_id#x] + : +- SubqueryAlias v + : +- Project [col1#x AS id#x, col2#x AS a_id#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias a + +- View (`a`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias v + +- Project [col1#x AS id#x] + +- LocalRelation [col1#x] + + +-- !query +create or replace temporary view innertab as select * from + (values (123L, 42L)) + as v(id, dat1) +-- !query analysis +CreateViewCommand `innertab`, select * from + (values (123L, 42L)) + as v(id, dat1), false, true, LocalTempView, true + +- Project [id#xL, dat1#xL] + +- SubqueryAlias v + +- Project [col1#xL AS id#xL, col2#xL AS dat1#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +SELECT * FROM + (SELECT udf(1) AS x) ss1 + LEFT JOIN + (SELECT udf(q1), udf(q2), udf(COALESCE(dat1, q1)) AS y + FROM int8_tbl LEFT JOIN innertab ON udf(udf(q2)) = id) ss2 + ON true +-- !query analysis +Project [x#x, udf(q1)#xL, udf(q2)#xL, y#xL] ++- Join LeftOuter, true + :- SubqueryAlias ss1 + : +- Project [cast(udf(cast(1 as string)) as int) AS x#x] + : +- OneRowRelation + +- SubqueryAlias ss2 + +- Project [cast(udf(cast(q1#xL as string)) as bigint) AS udf(q1)#xL, cast(udf(cast(q2#xL as string)) as bigint) AS udf(q2)#xL, cast(udf(cast(coalesce(dat1#xL, q1#xL) as string)) as bigint) AS y#xL] + +- Join LeftOuter, (cast(udf(cast(cast(udf(cast(q2#xL as string)) as bigint) as string)) as bigint) = id#xL) + :- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias innertab + +- View (`innertab`, [id#xL,dat1#xL]) + +- Project [cast(id#xL as bigint) AS id#xL, cast(dat1#xL as bigint) AS dat1#xL] + +- Project [id#xL, dat1#xL] + +- SubqueryAlias v + +- Project [col1#xL AS id#xL, col2#xL AS dat1#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(f1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`f1`", + "referenceNames" : "[`j`.`f1`, `j`.`f1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 87, + "stopIndex" : 88, + "fragment" : "f1" + } ] +} + + +-- !query +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(y.f1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`.`f1`", + "proposal" : "`j`.`f1`, `j`.`f1`, `x`.`q1`, `x`.`q2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 87, + "stopIndex" : 90, + "fragment" : "y.f1" + } ] +} + + +-- !query +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on udf(q1) = udf(udf(f1)) +-- !query analysis +Project [q1#xL, q2#xL, f1#x, ff#x] ++- Join Inner, (cast(udf(cast(q1#xL as string)) as bigint) = cast(cast(udf(cast(cast(udf(cast(f1#x as string)) as int) as string)) as int) as bigint)) + :- SubqueryAlias x + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias j + +- Join Cross + :- SubqueryAlias x + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias y + +- Project [f1#x AS ff#x] + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select udf(t1.uunique1) from + tenk1 t1 join tenk2 t2 on t1.two = udf(t2.two) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1`.`uunique1`", + "proposal" : "`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`hundred`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 22, + "fragment" : "t1.uunique1" + } ] +} + + +-- !query +select udf(udf(t2.uunique1)) from + tenk1 t1 join tenk2 t2 on udf(t1.two) = t2.two +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t2`.`uunique1`", + "proposal" : "`t2`.`unique1`, `t1`.`unique1`, `t2`.`unique2`, `t1`.`unique2`, `t2`.`hundred`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 26, + "fragment" : "t2.uunique1" + } ] +} + + +-- !query +select udf(uunique1) from + tenk1 t1 join tenk2 t2 on udf(t1.two) = udf(t2.two) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`uunique1`", + "proposal" : "`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`even`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 19, + "fragment" : "uunique1" + } ] +} + + +-- !query +select unique2, x.* +from tenk1 a, lateral (select * from int4_tbl b where f1 = a.unique1) x +-- !query analysis +Project [unique2#x, f1#x] ++- LateralJoin lateral-subquery#x [unique1#x], Inner + : +- SubqueryAlias x + : +- Project [f1#x] + : +- Filter (f1#x = outer(unique1#x)) + : +- SubqueryAlias b + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias a + +- SubqueryAlias spark_catalog.default.tenk1 + +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + + +-- !query +select unique2, x.* +from int4_tbl x, lateral (select unique2 from tenk1 where f1 = unique1) ss +-- !query analysis +Project [unique2#x, f1#x] ++- LateralJoin lateral-subquery#x [f1#x], Inner + : +- SubqueryAlias ss + : +- Project [unique2#x] + : +- Filter (outer(f1#x) = unique1#x) + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias x + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select unique2, x.* +from int4_tbl x left join lateral (select unique1, unique2 from tenk1 where f1 = unique1) ss on true +-- !query analysis +Project [unique2#x, f1#x] ++- LateralJoin lateral-subquery#x [f1#x], LeftOuter, true + : +- SubqueryAlias ss + : +- Project [unique1#x, unique2#x] + : +- Filter (outer(f1#x) = unique1#x) + : +- SubqueryAlias spark_catalog.default.tenk1 + : +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet + +- SubqueryAlias x + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from (select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1, + lateral (select x) ss2(y) +-- !query analysis +Project [x#x, f1#x, y#x] ++- LateralJoin lateral-subquery#x [x#x], Inner + : +- SubqueryAlias ss2 + : +- Project [x#x AS y#x] + : +- Project [outer(x#x)] + : +- OneRowRelation + +- Join Inner, (x#x = cast(f1#x as double)) + :- SubqueryAlias ss1 + : +- Project [(cast(f1#x as double) / cast(2 as double)) AS x#x] + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias i4 + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from ((select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1) j, + lateral (select x) ss2(y) +-- !query analysis +Project [x#x, f1#x, y#x] ++- LateralJoin lateral-subquery#x [x#x], Inner + : +- SubqueryAlias ss2 + : +- Project [x#x AS y#x] + : +- Project [outer(x#x)] + : +- OneRowRelation + +- SubqueryAlias j + +- Join Inner, (x#x = cast(f1#x as double)) + :- SubqueryAlias ss1 + : +- Project [(cast(f1#x as double) / cast(2 as double)) AS x#x] + : +- SubqueryAlias int4_tbl + : +- View (`INT4_TBL`, [f1#x]) + : +- Project [cast(f1#x as int) AS f1#x] + : +- Project [f1#x] + : +- SubqueryAlias v + : +- Project [col1#x AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias i4 + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from + int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, + lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2) +-- !query analysis +Project [q1#xL, q2#xL, q1#xL, q2#xL, xq1#xL, yq1#xL, yq2#xL] ++- LateralJoin lateral-subquery#x [q1#xL && q1#xL && q2#xL], Inner + : +- SubqueryAlias v + : +- Project [q1#xL AS xq1#xL, q1#xL AS yq1#xL, q2#xL AS yq2#xL] + : +- Project [outer(q1#xL), outer(q1#xL), outer(q2#xL)] + : +- OneRowRelation + +- Join LeftOuter, (q2#xL = q1#xL) + :- SubqueryAlias x + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias y + +- Project [q1#xL, coalesce(q2#xL, cast(0 as bigint)) AS q2#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select x.* from + int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, + lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2) +-- !query analysis +Project [q1#xL, q2#xL] ++- LateralJoin lateral-subquery#x [q1#xL && q1#xL && q2#xL], Inner + : +- SubqueryAlias v + : +- Project [q1#xL AS xq1#xL, q1#xL AS yq1#xL, q2#xL AS yq2#xL] + : +- Project [outer(q1#xL), outer(q1#xL), outer(q2#xL)] + : +- OneRowRelation + +- Join LeftOuter, (q2#xL = q1#xL) + :- SubqueryAlias x + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias y + +- Project [q1#xL, coalesce(q2#xL, cast(0 as bigint)) AS q2#xL] + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select * from + int8_tbl a left join + lateral (select *, a.q2 as x from int8_tbl b) ss on a.q2 = ss.q1 +-- !query analysis +Project [q1#xL, q2#xL, q1#xL, q2#xL, x#xL] ++- LateralJoin lateral-subquery#x [q2#xL], LeftOuter, (q2#xL = q1#xL) + : +- SubqueryAlias ss + : +- Project [q1#xL, q2#xL, outer(q2#xL) AS x#xL] + : +- SubqueryAlias b + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias a + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select * from + int8_tbl a left join + lateral (select *, coalesce(a.q2, 42) as x from int8_tbl b) ss on a.q2 = ss.q1 +-- !query analysis +Project [q1#xL, q2#xL, q1#xL, q2#xL, x#xL] ++- LateralJoin lateral-subquery#x [q2#xL], LeftOuter, (q2#xL = q1#xL) + : +- SubqueryAlias ss + : +- Project [q1#xL, q2#xL, coalesce(outer(q2#xL), cast(42 as bigint)) AS x#xL] + : +- SubqueryAlias b + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias a + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select * from int4_tbl i left join + lateral (select * from int2_tbl j where i.f1 = j.f1) k on true +-- !query analysis +Project [f1#x, f1#x] ++- LateralJoin lateral-subquery#x [f1#x], LeftOuter, true + : +- SubqueryAlias k + : +- Project [f1#x] + : +- Filter (outer(f1#x) = cast(f1#x as int)) + : +- SubqueryAlias j + : +- SubqueryAlias int2_tbl + : +- View (`INT2_TBL`, [f1#x]) + : +- Project [cast(col1#x as smallint) AS f1#x] + : +- LocalRelation [col1#x] + +- SubqueryAlias i + +- SubqueryAlias int4_tbl + +- View (`INT4_TBL`, [f1#x]) + +- Project [cast(f1#x as int) AS f1#x] + +- Project [f1#x] + +- SubqueryAlias v + +- Project [col1#x AS f1#x] + +- LocalRelation [col1#x] + + +-- !query +select * from + int8_tbl a left join lateral + (select b.q1 as bq1, c.q1 as cq1, least(a.q1,b.q1,c.q1) from + int8_tbl b cross join int8_tbl c) ss + on a.q2 = ss.bq1 +-- !query analysis +Project [q1#xL, q2#xL, bq1#xL, cq1#xL, least(outer(a.q1), q1, q1)#xL] ++- LateralJoin lateral-subquery#x [q1#xL], LeftOuter, (q2#xL = bq1#xL) + : +- SubqueryAlias ss + : +- Project [q1#xL AS bq1#xL, q1#xL AS cq1#xL, least(outer(q1#xL), q1#xL, q1#xL) AS least(outer(a.q1), q1, q1)#xL] + : +- Join Cross + : :- SubqueryAlias b + : : +- SubqueryAlias int8_tbl + : : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : : +- Project [q1#xL, q2#xL] + : : +- SubqueryAlias v + : : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : : +- LocalRelation [col1#xL, col2#xL] + : +- SubqueryAlias c + : +- SubqueryAlias int8_tbl + : +- View (`INT8_TBL`, [q1#xL,q2#xL]) + : +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + : +- Project [q1#xL, q2#xL] + : +- SubqueryAlias v + : +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + : +- LocalRelation [col1#xL, col2#xL] + +- SubqueryAlias a + +- SubqueryAlias int8_tbl + +- View (`INT8_TBL`, [q1#xL,q2#xL]) + +- Project [cast(q1#xL as bigint) AS q1#xL, cast(q2#xL as bigint) AS q2#xL] + +- Project [q1#xL, q2#xL] + +- SubqueryAlias v + +- Project [col1#xL AS q1#xL, col2#xL AS q2#xL] + +- LocalRelation [col1#xL, col2#xL] + + +-- !query +select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 56, + "stopIndex" : 57, + "fragment" : "f1" + } ] +} + + +-- !query +select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`.`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 43, + "stopIndex" : 46, + "fragment" : "a.f1" + } ] +} + + +-- !query +select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 63, + "fragment" : "f1" + } ] +} + + +-- !query +select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`.`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 61, + "stopIndex" : 64, + "fragment" : "a.f1" + } ] +} + + +-- !query +CREATE TABLE j1 (id1 int, id2 int) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`j1`, false + + +-- !query +CREATE TABLE j2 (id1 int, id2 int) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`j2`, false + + +-- !query +INSERT INTO j1 values(1,1),(1,2) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1], Append, `spark_catalog`.`default`.`j1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1), [id1, id2] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO j2 values(1,1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2], Append, `spark_catalog`.`default`.`j2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2), [id1, id2] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +INSERT INTO j2 values(1,2) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2], Append, `spark_catalog`.`default`.`j2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2), [id1, id2] ++- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from j1 +inner join j2 on udf(j1.id1) = udf(j2.id1) and udf(udf(j1.id2)) = udf(j2.id2) +where udf(j1.id1) % 1000 = 1 and udf(udf(j2.id1) % 1000) = 1 +-- !query analysis +Project [id1#x, id2#x, id1#x, id2#x] ++- Filter (((cast(udf(cast(id1#x as string)) as int) % 1000) = 1) AND (cast(udf(cast((cast(udf(cast(id1#x as string)) as int) % 1000) as string)) as int) = 1)) + +- Join Inner, ((cast(udf(cast(id1#x as string)) as int) = cast(udf(cast(id1#x as string)) as int)) AND (cast(udf(cast(cast(udf(cast(id2#x as string)) as int) as string)) as int) = cast(udf(cast(id2#x as string)) as int))) + :- SubqueryAlias spark_catalog.default.j1 + : +- Relation spark_catalog.default.j1[id1#x,id2#x] parquet + +- SubqueryAlias spark_catalog.default.j2 + +- Relation spark_catalog.default.j2[id1#x,id2#x] parquet + + +-- !query +drop table j1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`j1`, false, false, false + + +-- !query +drop table j2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`j2`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_having.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_having.sql.out new file mode 100644 index 0000000000000..b864765b0e817 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_having.sql.out @@ -0,0 +1,215 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE test_having (a int, b int, c string, d string) USING parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`test_having`, false + + +-- !query +INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (1, 2, 'AAAA', 'b') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (2, 2, 'AAAA', 'c') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (3, 3, 'BBBB', 'D') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (4, 3, 'BBBB', 'e') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (5, 3, 'bbbb', 'F') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (6, 4, 'cccc', 'g') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (7, 4, 'cccc', 'h') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (8, 4, 'CCCC', 'I') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_having VALUES (9, 4, 'CCCC', 'j') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +SELECT udf(b), udf(c) FROM test_having + GROUP BY b, c HAVING udf(count(*)) = 1 ORDER BY udf(b), udf(c) +-- !query analysis +Project [udf(b)#x, udf(c)#x] ++- Sort [cast(udf(cast(b#x as string)) as int) ASC NULLS FIRST, cast(udf(cast(c#x as string)) as string) ASC NULLS FIRST], true + +- Project [udf(b)#x, udf(c)#x, b#x, c#x] + +- Filter (cast(udf(cast(count(1)#xL as string)) as bigint) = cast(1 as bigint)) + +- Aggregate [b#x, c#x], [cast(udf(cast(b#x as string)) as int) AS udf(b)#x, cast(udf(cast(c#x as string)) as string) AS udf(c)#x, count(1) AS count(1)#xL, b#x, c#x] + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(b), udf(c) FROM test_having + GROUP BY b, c HAVING udf(b) = 3 ORDER BY udf(b), udf(c) +-- !query analysis +Project [udf(b)#x, udf(c)#x] ++- Sort [cast(udf(cast(b#x as string)) as int) ASC NULLS FIRST, cast(udf(cast(c#x as string)) as string) ASC NULLS FIRST], true + +- Filter (udf(b)#x = 3) + +- Aggregate [b#x, c#x], [cast(udf(cast(b#x as string)) as int) AS udf(b)#x, cast(udf(cast(c#x as string)) as string) AS udf(c)#x, b#x, c#x] + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(c), max(udf(a)) FROM test_having + GROUP BY c HAVING udf(count(*)) > 2 OR udf(min(a)) = udf(max(a)) + ORDER BY c +-- !query analysis +Project [udf(c)#x, max(udf(a))#x] ++- Sort [c#x ASC NULLS FIRST], true + +- Project [udf(c)#x, max(udf(a))#x, c#x] + +- Filter ((cast(udf(cast(count(1)#xL as string)) as bigint) > cast(2 as bigint)) OR (cast(udf(cast(min(a#x)#x as string)) as int) = cast(udf(cast(max(a#x)#x as string)) as int))) + +- Aggregate [c#x], [cast(udf(cast(c#x as string)) as string) AS udf(c)#x, max(cast(udf(cast(a#x as string)) as int)) AS max(udf(a))#x, count(1) AS count(1)#xL, min(a#x) AS min(a#x)#x, max(a#x) AS max(a#x)#x, c#x] + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(udf(min(udf(a)))), udf(udf(max(udf(a)))) FROM test_having HAVING udf(udf(min(udf(a)))) = udf(udf(max(udf(a)))) +-- !query analysis +Filter (udf(udf(min(udf(a))))#x = udf(udf(max(udf(a))))#x) ++- Aggregate [cast(udf(cast(cast(udf(cast(min(cast(udf(cast(a#x as string)) as int)) as string)) as int) as string)) as int) AS udf(udf(min(udf(a))))#x, cast(udf(cast(cast(udf(cast(max(cast(udf(cast(a#x as string)) as int)) as string)) as int) as string)) as int) AS udf(udf(max(udf(a))))#x] + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(min(udf(a))), udf(udf(max(a))) FROM test_having HAVING udf(min(a)) < udf(max(udf(a))) +-- !query analysis +Project [udf(min(udf(a)))#x, udf(udf(max(a)))#x] ++- Filter (cast(udf(cast(min(a#x)#x as string)) as int) < cast(udf(cast(max(cast(udf(cast(a#x as string)) as int))#x as string)) as int)) + +- Aggregate [cast(udf(cast(min(cast(udf(cast(a#x as string)) as int)) as string)) as int) AS udf(min(udf(a)))#x, cast(udf(cast(cast(udf(cast(max(a#x) as string)) as int) as string)) as int) AS udf(udf(max(a)))#x, min(a#x) AS min(a#x)#x, max(cast(udf(cast(a#x as string)) as int)) AS max(cast(udf(cast(a#x as string)) as int))#x] + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(a) FROM test_having HAVING udf(min(a)) < udf(max(a)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "SELECT udf(a) FROM test_having HAVING udf(min(a)) < udf(max(a))" + } ] +} + + +-- !query +SELECT 1 AS one FROM test_having HAVING udf(a) > 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`", + "proposal" : "`one`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 45, + "fragment" : "a" + } ] +} + + +-- !query +SELECT 1 AS one FROM test_having HAVING udf(udf(1) > udf(2)) +-- !query analysis +Filter cast(cast(udf(cast((cast(udf(cast(one#x as string)) as int) > cast(udf(cast(2 as string)) as int)) as string)) as boolean) as boolean) ++- Aggregate [1 AS one#x] + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT 1 AS one FROM test_having HAVING udf(udf(1) < udf(2)) +-- !query analysis +Filter cast(cast(udf(cast((cast(udf(cast(one#x as string)) as int) < cast(udf(cast(2 as string)) as int)) as string)) as boolean) as boolean) ++- Aggregate [1 AS one#x] + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT 1 AS one FROM test_having WHERE 1/udf(a) = 1 HAVING 1 < 2 +-- !query analysis +Filter (one#x < 2) ++- Aggregate [1 AS one#x] + +- Filter ((cast(1 as double) / cast(cast(udf(cast(a#x as string)) as int) as double)) = cast(1 as double)) + +- SubqueryAlias spark_catalog.default.test_having + +- Relation spark_catalog.default.test_having[a#x,b#x,c#x,d#x] parquet + + +-- !query +DROP TABLE test_having +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`test_having`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_implicit.sql.out new file mode 100644 index 0000000000000..5ce91e5690ac8 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_implicit.sql.out @@ -0,0 +1,452 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE test_missing_target (a int, b int, c string, d string) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`test_missing_target`, false + + +-- !query +INSERT INTO test_missing_target VALUES (0, 1, 'XXXX', 'A') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (1, 2, 'ABAB', 'b') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (2, 2, 'ABAB', 'c') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (3, 3, 'BBBB', 'D') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (4, 3, 'BBBB', 'e') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (5, 3, 'bbbb', 'F') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (6, 4, 'cccc', 'g') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (7, 4, 'cccc', 'h') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (8, 4, 'CCCC', 'I') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +INSERT INTO test_missing_target VALUES (9, 4, 'CCCC', 'j') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x] + +- LocalRelation [col1#x, col2#x, col3#x, col4#x] + + +-- !query +SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY +udf(test_missing_target.c) +ORDER BY udf(c) +-- !query analysis +Sort [udf(c)#x ASC NULLS FIRST], true ++- Aggregate [cast(udf(cast(c#x as string)) as string)], [cast(udf(cast(c#x as string)) as string) AS udf(c)#x, cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(test_missing_target.c) +ORDER BY udf(c) +-- !query analysis +Project [udf(count(1))#xL] ++- Sort [cast(udf(cast(c#x as string)) as string)#x ASC NULLS FIRST], true + +- Aggregate [cast(udf(cast(c#x as string)) as string)], [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL, cast(udf(cast(c#x as string)) as string) AS cast(udf(cast(c#x as string)) as string)#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`b`", + "proposal" : "`udf(count(1))`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 76, + "stopIndex" : 76, + "fragment" : "b" + } ] +} + + +-- !query +SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(b) ORDER BY udf(b) +-- !query analysis +Project [udf(count(1))#xL] ++- Sort [cast(udf(cast(b#x as string)) as int)#x ASC NULLS FIRST], true + +- Aggregate [cast(udf(cast(b#x as string)) as int)], [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL, cast(udf(cast(b#x as string)) as int) AS cast(udf(cast(b#x as string)) as int)#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(test_missing_target.b), udf(count(*)) + FROM test_missing_target GROUP BY udf(b) ORDER BY udf(b) +-- !query analysis +Sort [udf(b)#x ASC NULLS FIRST], true ++- Aggregate [cast(udf(cast(b#x as string)) as int)], [cast(udf(cast(b#x as string)) as int) AS udf(b)#x, cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(c) FROM test_missing_target ORDER BY udf(a) +-- !query analysis +Project [udf(c)#x] ++- Sort [cast(udf(cast(a#x as string)) as int) ASC NULLS FIRST], true + +- Project [cast(udf(cast(c#x as string)) as string) AS udf(c)#x, a#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(b) ORDER BY udf(b) desc +-- !query analysis +Project [udf(count(1))#xL] ++- Sort [cast(udf(cast(b#x as string)) as int)#x DESC NULLS LAST], true + +- Aggregate [cast(udf(cast(b#x as string)) as int)], [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL, cast(udf(cast(b#x as string)) as int) AS cast(udf(cast(b#x as string)) as int)#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(count(*)) FROM test_missing_target ORDER BY udf(1) desc +-- !query analysis +Sort [cast(udf(cast(1 as string)) as int) DESC NULLS LAST], true ++- Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY 1 ORDER BY 1 +-- !query analysis +Sort [udf(c)#x ASC NULLS FIRST], true ++- Aggregate [cast(udf(cast(c#x as string)) as string)], [cast(udf(cast(c#x as string)) as string) AS udf(c)#x, cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY 3 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "3", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 64, + "stopIndex" : 64, + "fragment" : "3" + } ] +} + + +-- !query +SELECT udf(count(*)) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(b) ORDER BY udf(b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`b`", + "referenceNames" : "[`x`.`b`, `y`.`b`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 113, + "stopIndex" : 113, + "fragment" : "b" + } ] +} + + +-- !query +SELECT udf(a), udf(a) FROM test_missing_target + ORDER BY udf(a) +-- !query analysis +Project [udf(a)#x, udf(a)#x] ++- Sort [cast(udf(cast(a#x as string)) as int) ASC NULLS FIRST], true + +- Project [cast(udf(cast(a#x as string)) as int) AS udf(a)#x, cast(udf(cast(a#x as string)) as int) AS udf(a)#x, a#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(udf(a)/2), udf(udf(a)/2) FROM test_missing_target + ORDER BY udf(udf(a)/2) +-- !query analysis +Project [udf((udf(a) / 2))#x, udf((udf(a) / 2))#x] ++- Sort [cast(udf(cast((cast(cast(udf(cast(a#x as string)) as int) as double) / cast(2 as double)) as string)) as double) ASC NULLS FIRST], true + +- Project [cast(udf(cast((cast(cast(udf(cast(a#x as string)) as int) as double) / cast(2 as double)) as string)) as double) AS udf((udf(a) / 2))#x, cast(udf(cast((cast(cast(udf(cast(a#x as string)) as int) as double) / cast(2 as double)) as string)) as double) AS udf((udf(a) / 2))#x, a#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(a/2), udf(a/2) FROM test_missing_target + GROUP BY udf(a/2) ORDER BY udf(a/2) +-- !query analysis +Sort [udf((a / 2))#x ASC NULLS FIRST], true ++- Aggregate [cast(udf(cast((cast(a#x as double) / cast(2 as double)) as string)) as double)], [cast(udf(cast((cast(a#x as double) / cast(2 as double)) as string)) as double) AS udf((a / 2))#x, cast(udf(cast((cast(a#x as double) / cast(2 as double)) as string)) as double) AS udf((a / 2))#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(x.b), udf(count(*)) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(x.b) ORDER BY udf(x.b) +-- !query analysis +Sort [udf(b)#x ASC NULLS FIRST], true ++- Aggregate [cast(udf(cast(b#x as string)) as int)], [cast(udf(cast(b#x as string)) as int) AS udf(b)#x, cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] + +- Filter (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(a#x as string)) as int)) + +- Join Inner + :- SubqueryAlias x + : +- SubqueryAlias spark_catalog.default.test_missing_target + : +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + +- SubqueryAlias y + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(count(*)) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(x.b) ORDER BY udf(x.b) +-- !query analysis +Project [udf(count(1))#xL] ++- Sort [cast(udf(cast(b#x as string)) as int)#x ASC NULLS FIRST], true + +- Aggregate [cast(udf(cast(b#x as string)) as int)], [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL, cast(udf(cast(b#x as string)) as int) AS cast(udf(cast(b#x as string)) as int)#x] + +- Filter (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(a#x as string)) as int)) + +- Join Inner + :- SubqueryAlias x + : +- SubqueryAlias spark_catalog.default.test_missing_target + : +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + +- SubqueryAlias y + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(a%2), udf(count(udf(b))) FROM test_missing_target +GROUP BY udf(test_missing_target.a%2) +ORDER BY udf(test_missing_target.a%2) +-- !query analysis +Sort [udf((a % 2))#x ASC NULLS FIRST], true ++- Aggregate [cast(udf(cast((a#x % 2) as string)) as int)], [cast(udf(cast((a#x % 2) as string)) as int) AS udf((a % 2))#x, cast(udf(cast(count(cast(udf(cast(b#x as string)) as int)) as string)) as bigint) AS udf(count(udf(b)))#xL] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(count(c)) FROM test_missing_target +GROUP BY udf(lower(test_missing_target.c)) +ORDER BY udf(lower(test_missing_target.c)) +-- !query analysis +Project [udf(count(c))#xL] ++- Sort [cast(udf(cast(lower(c#x) as string)) as string)#x ASC NULLS FIRST], true + +- Aggregate [cast(udf(cast(lower(c#x) as string)) as string)], [cast(udf(cast(count(c#x) as string)) as bigint) AS udf(count(c))#xL, cast(udf(cast(lower(c#x) as string)) as string) AS cast(udf(cast(lower(c#x) as string)) as string)#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(count(udf(a))) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`b`", + "proposal" : "`udf(count(udf(a)))`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 81, + "stopIndex" : 81, + "fragment" : "b" + } ] +} + + +-- !query +SELECT udf(count(b)) FROM test_missing_target GROUP BY udf(b/2) ORDER BY udf(b/2) +-- !query analysis +Project [udf(count(b))#xL] ++- Sort [cast(udf(cast((cast(b#x as double) / cast(2 as double)) as string)) as double)#x ASC NULLS FIRST], true + +- Aggregate [cast(udf(cast((cast(b#x as double) / cast(2 as double)) as string)) as double)], [cast(udf(cast(count(b#x) as string)) as bigint) AS udf(count(b))#xL, cast(udf(cast((cast(b#x as double) / cast(2 as double)) as string)) as double) AS cast(udf(cast((cast(b#x as double) / cast(2 as double)) as string)) as double)#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(lower(test_missing_target.c)), udf(count(udf(c))) + FROM test_missing_target GROUP BY udf(lower(c)) ORDER BY udf(lower(c)) +-- !query analysis +Sort [udf(lower(c))#x ASC NULLS FIRST], true ++- Aggregate [cast(udf(cast(lower(c#x) as string)) as string)], [cast(udf(cast(lower(c#x) as string)) as string) AS udf(lower(c))#x, cast(udf(cast(count(cast(udf(cast(c#x as string)) as string)) as string)) as bigint) AS udf(count(udf(c)))#xL] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(a) FROM test_missing_target ORDER BY udf(upper(udf(d))) +-- !query analysis +Project [udf(a)#x] ++- Sort [cast(udf(cast(upper(cast(udf(cast(d#x as string)) as string)) as string)) as string) ASC NULLS FIRST], true + +- Project [cast(udf(cast(a#x as string)) as int) AS udf(a)#x, d#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(count(b)) FROM test_missing_target + GROUP BY udf((b + 1) / 2) ORDER BY udf((b + 1) / 2) desc +-- !query analysis +Project [udf(count(b))#xL] ++- Sort [cast(udf(cast((cast((b#x + 1) as double) / cast(2 as double)) as string)) as double)#x DESC NULLS LAST], true + +- Aggregate [cast(udf(cast((cast((b#x + 1) as double) / cast(2 as double)) as string)) as double)], [cast(udf(cast(count(b#x) as string)) as bigint) AS udf(count(b))#xL, cast(udf(cast((cast((b#x + 1) as double) / cast(2 as double)) as string)) as double) AS cast(udf(cast((cast((b#x + 1) as double) / cast(2 as double)) as string)) as double)#x] + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(count(udf(x.a))) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(b/2) ORDER BY udf(b/2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`b`", + "referenceNames" : "[`x`.`b`, `y`.`b`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 120, + "stopIndex" : 120, + "fragment" : "b" + } ] +} + + +-- !query +SELECT udf(x.b/2), udf(count(udf(x.b))) FROM test_missing_target x, +test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(x.b/2) ORDER BY udf(x.b/2) +-- !query analysis +Sort [udf((b / 2))#x ASC NULLS FIRST], true ++- Aggregate [cast(udf(cast((cast(b#x as double) / cast(2 as double)) as string)) as double)], [cast(udf(cast((cast(b#x as double) / cast(2 as double)) as string)) as double) AS udf((b / 2))#x, cast(udf(cast(count(cast(udf(cast(b#x as string)) as int)) as string)) as bigint) AS udf(count(udf(b)))#xL] + +- Filter (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(a#x as string)) as int)) + +- Join Inner + :- SubqueryAlias x + : +- SubqueryAlias spark_catalog.default.test_missing_target + : +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + +- SubqueryAlias y + +- SubqueryAlias spark_catalog.default.test_missing_target + +- Relation spark_catalog.default.test_missing_target[a#x,b#x,c#x,d#x] parquet + + +-- !query +SELECT udf(count(udf(b))) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(x.b/2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`b`", + "referenceNames" : "[`x`.`b`, `y`.`b`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 22, + "fragment" : "b" + } ] +} + + +-- !query +DROP TABLE test_missing_target +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`test_missing_target`, false, false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-count.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-count.sql.out new file mode 100644 index 0000000000000..90b488a41d5d0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-count.sql.out @@ -0,0 +1,71 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b) +-- !query analysis +CreateViewCommand `testData`, SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b), false, true, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT + udf(count(*)), udf(count(1)), udf(count(null)), udf(count(a)), udf(count(b)), udf(count(a + b)), udf(count((a, b))) +FROM testData +-- !query analysis +Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL, cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL, cast(udf(cast(count(null) as string)) as bigint) AS udf(count(NULL))#xL, cast(udf(cast(count(a#x) as string)) as bigint) AS udf(count(a))#xL, cast(udf(cast(count(b#x) as string)) as bigint) AS udf(count(b))#xL, cast(udf(cast(count((a#x + b#x)) as string)) as bigint) AS udf(count((a + b)))#xL, cast(udf(cast(count(named_struct(a, a#x, b, b#x)) as string)) as bigint) AS udf(count(named_struct(a, a, b, b)))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT + udf(count(DISTINCT 1)), + udf(count(DISTINCT null)), + udf(count(DISTINCT a)), + udf(count(DISTINCT b)), + udf(count(DISTINCT (a + b))), + udf(count(DISTINCT (a, b))) +FROM testData +-- !query analysis +Aggregate [cast(udf(cast(count(distinct 1) as string)) as bigint) AS udf(count(DISTINCT 1))#xL, cast(udf(cast(count(distinct null) as string)) as bigint) AS udf(count(DISTINCT NULL))#xL, cast(udf(cast(count(distinct a#x) as string)) as bigint) AS udf(count(DISTINCT a))#xL, cast(udf(cast(count(distinct b#x) as string)) as bigint) AS udf(count(DISTINCT b))#xL, cast(udf(cast(count(distinct (a#x + b#x)) as string)) as bigint) AS udf(count(DISTINCT (a + b)))#xL, cast(udf(cast(count(distinct named_struct(a, a#x, b, b#x)) as string)) as bigint) AS udf(count(DISTINCT named_struct(a, a, b, b)))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT udf(count(a, b)), udf(count(b, a)), udf(count(testData.*, testData.*)) FROM testData +-- !query analysis +Aggregate [cast(udf(cast(count(a#x, b#x) as string)) as bigint) AS udf(count(a, b))#xL, cast(udf(cast(count(b#x, a#x) as string)) as bigint) AS udf(count(b, a))#xL, cast(udf(cast(count(a#x, b#x, a#x, b#x) as string)) as bigint) AS udf(count(a, b, a, b))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT + udf(count(DISTINCT a, b)), udf(count(DISTINCT b, a)), udf(count(DISTINCT *)), udf(count(DISTINCT testData.*, testData.*)) +FROM testData +-- !query analysis +Aggregate [cast(udf(cast(count(distinct a#x, b#x) as string)) as bigint) AS udf(count(DISTINCT a, b))#xL, cast(udf(cast(count(distinct b#x, a#x) as string)) as bigint) AS udf(count(DISTINCT b, a))#xL, cast(udf(cast(count(distinct a#x, b#x) as string)) as bigint) AS udf(count(DISTINCT a, b))#xL, cast(udf(cast(count(distinct a#x, b#x, a#x, b#x) as string)) as bigint) AS udf(count(DISTINCT a, b, a, b))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] 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 new file mode 100644 index 0000000000000..4568744e90718 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-cross-join.sql.out @@ -0,0 +1,259 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query analysis +CreateViewCommand `nt1`, select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1), false, false, LocalTempView, true + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- LocalRelation [k#x, v1#x] + + +-- !query +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query analysis +CreateViewCommand `nt2`, select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2), false, false, LocalTempView, true + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 cross join nt2 +-- !query analysis +Project [k#x, v1#x, k#x, v2#x] ++- Join Cross + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 cross join nt2 where udf(nt1.k) = udf(nt2.k) +-- !query analysis +Project [k#x, v1#x, k#x, v2#x] ++- Filter (cast(udf(cast(k#x as string)) as string) = cast(udf(cast(k#x as string)) as string)) + +- Join Cross + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 cross join nt2 on (udf(nt1.k) = udf(nt2.k)) +-- !query analysis +Project [k#x, v1#x, k#x, v2#x] ++- Join Cross, (cast(udf(cast(k#x as string)) as string) = cast(udf(cast(k#x as string)) as string)) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +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))) + +- Join Cross + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT udf(a.key), udf(b.key) FROM +(SELECT udf(k) key FROM nt1 WHERE v1 < 2) a +CROSS JOIN +(SELECT udf(k) key FROM nt2 WHERE v2 = 22) b +-- !query analysis +Project [cast(udf(cast(key#x as string)) as string) AS udf(key)#x, cast(udf(cast(key#x as string)) as string) AS udf(key)#x] ++- Join Cross + :- SubqueryAlias a + : +- Project [cast(udf(cast(k#x as string)) as string) AS key#x] + : +- Filter (v1#x < 2) + : +- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias b + +- Project [cast(udf(cast(k#x as string)) as string) AS key#x] + +- Filter (v2#x = 22) + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +create temporary view A(a, va) as select * from nt1 +-- !query analysis +CreateViewCommand `A`, [(a,None), (va,None)], select * from nt1, false, false, LocalTempView, true + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- View (`nt1`, [k#x,v1#x]) + +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- LocalRelation [k#x, v1#x] + + +-- !query +create temporary view B(b, vb) as select * from nt1 +-- !query analysis +CreateViewCommand `B`, [(b,None), (vb,None)], select * from nt1, false, false, LocalTempView, true + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- View (`nt1`, [k#x,v1#x]) + +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- LocalRelation [k#x, v1#x] + + +-- !query +create temporary view C(c, vc) as select * from nt1 +-- !query analysis +CreateViewCommand `C`, [(c,None), (vc,None)], select * from nt1, false, false, LocalTempView, true + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- View (`nt1`, [k#x,v1#x]) + +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- LocalRelation [k#x, v1#x] + + +-- !query +create temporary view D(d, vd) as select * from nt1 +-- !query analysis +CreateViewCommand `D`, [(d,None), (vd,None)], select * from nt1, false, false, LocalTempView, true + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- View (`nt1`, [k#x,v1#x]) + +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- LocalRelation [k#x, v1#x] + + +-- !query +select * from ((A join B on (udf(a) = udf(b))) cross join C) join D on (udf(a) = udf(d)) +-- !query analysis +Project [a#x, va#x, b#x, vb#x, c#x, vc#x, d#x, vd#x] ++- Join Inner, (cast(udf(cast(a#x as string)) as string) = cast(udf(cast(d#x as string)) as string)) + :- Join Cross + : :- Join Inner, (cast(udf(cast(a#x as string)) as string) = cast(udf(cast(b#x as string)) as string)) + : : :- SubqueryAlias a + : : : +- View (`A`, [a#x,va#x]) + : : : +- Project [cast(k#x as string) AS a#x, cast(v1#x as int) AS va#x] + : : : +- Project [k#x, v1#x] + : : : +- SubqueryAlias nt1 + : : : +- View (`nt1`, [k#x,v1#x]) + : : : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : : : +- Project [k#x, v1#x] + : : : +- SubqueryAlias nt1 + : : : +- LocalRelation [k#x, v1#x] + : : +- SubqueryAlias b + : : +- View (`B`, [b#x,vb#x]) + : : +- Project [cast(k#x as string) AS b#x, cast(v1#x as int) AS vb#x] + : : +- Project [k#x, v1#x] + : : +- SubqueryAlias nt1 + : : +- View (`nt1`, [k#x,v1#x]) + : : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : : +- Project [k#x, v1#x] + : : +- SubqueryAlias nt1 + : : +- LocalRelation [k#x, v1#x] + : +- SubqueryAlias c + : +- View (`C`, [c#x,vc#x]) + : +- Project [cast(k#x as string) AS c#x, cast(v1#x as int) AS vc#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias d + +- View (`D`, [d#x,vd#x]) + +- Project [cast(k#x as string) AS d#x, cast(v1#x as int) AS vd#x] + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- View (`nt1`, [k#x,v1#x]) + +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- LocalRelation [k#x, v1#x] + + +-- !query +SELECT * FROM nt1 CROSS JOIN nt2 ON (udf(nt1.k) > udf(nt2.k)) +-- !query analysis +Project [k#x, v1#x, k#x, v2#x] ++- Join Cross, (cast(udf(cast(k#x as string)) as string) > cast(udf(cast(k#x as string)) as string)) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-except-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-except-all.sql.out new file mode 100644 index 0000000000000..9787bdcc9c352 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-except-all.sql.out @@ -0,0 +1,670 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1) +-- !query analysis +CreateViewCommand `tab1`, SELECT * FROM VALUES + (0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1), false, false, LocalTempView, true + +- Project [c1#x] + +- SubqueryAlias tab1 + +- LocalRelation [c1#x] + + +-- !query +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1), (2), (2), (3), (5), (5), (null) AS tab2(c1) +-- !query analysis +CreateViewCommand `tab2`, SELECT * FROM VALUES + (1), (2), (2), (3), (5), (5), (null) AS tab2(c1), false, false, LocalTempView, true + +- Project [c1#x] + +- SubqueryAlias tab2 + +- LocalRelation [c1#x] + + +-- !query +CREATE TEMPORARY VIEW tab3 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (2, 3), + (2, 2) + AS tab3(k, v) +-- !query analysis +CreateViewCommand `tab3`, SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (2, 3), + (2, 2) + AS tab3(k, v), false, false, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias tab3 + +- LocalRelation [k#x, v#x] + + +-- !query +CREATE TEMPORARY VIEW tab4 AS SELECT * FROM VALUES + (1, 2), + (2, 3), + (2, 2), + (2, 2), + (2, 20) + AS tab4(k, v) +-- !query analysis +CreateViewCommand `tab4`, SELECT * FROM VALUES + (1, 2), + (2, 3), + (2, 2), + (2, 2), + (2, 20) + AS tab4(k, v), false, false, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 +-- !query analysis +Except All true +:- Project [cast(udf(cast(c1#x as string)) as int) AS udf(c1)#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [c1#x]) +: +- Project [cast(c1#x as int) AS c1#x] +: +- Project [c1#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [c1#x] ++- Project [cast(udf(cast(c1#x as string)) as int) AS udf(c1)#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [c1#x]) + +- Project [cast(c1#x as int) AS c1#x] + +- Project [c1#x] + +- SubqueryAlias tab2 + +- LocalRelation [c1#x] + + +-- !query +SELECT udf(c1) FROM tab1 +MINUS ALL +SELECT udf(c1) FROM tab2 +-- !query analysis +Except All true +:- Project [cast(udf(cast(c1#x as string)) as int) AS udf(c1)#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [c1#x]) +: +- Project [cast(c1#x as int) AS c1#x] +: +- Project [c1#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [c1#x] ++- Project [cast(udf(cast(c1#x as string)) as int) AS udf(c1)#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [c1#x]) + +- Project [cast(c1#x as int) AS c1#x] + +- Project [c1#x] + +- SubqueryAlias tab2 + +- LocalRelation [c1#x] + + +-- !query +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1) IS NOT NULL +-- !query analysis +Except All true +:- Project [cast(udf(cast(c1#x as string)) as int) AS udf(c1)#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [c1#x]) +: +- Project [cast(c1#x as int) AS c1#x] +: +- Project [c1#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [c1#x] ++- Project [cast(udf(cast(c1#x as string)) as int) AS udf(c1)#x] + +- Filter isnotnull(cast(udf(cast(c1#x as string)) as int)) + +- SubqueryAlias tab2 + +- View (`tab2`, [c1#x]) + +- Project [cast(c1#x as int) AS c1#x] + +- Project [c1#x] + +- SubqueryAlias tab2 + +- LocalRelation [c1#x] + + +-- !query +SELECT udf(c1) FROM tab1 WHERE udf(c1) > 5 +EXCEPT ALL +SELECT udf(c1) FROM tab2 +-- !query analysis +Except All true +:- Project [cast(udf(cast(c1#x as string)) as int) AS udf(c1)#x] +: +- Filter (cast(udf(cast(c1#x as string)) as int) > 5) +: +- SubqueryAlias tab1 +: +- View (`tab1`, [c1#x]) +: +- Project [cast(c1#x as int) AS c1#x] +: +- Project [c1#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [c1#x] ++- Project [cast(udf(cast(c1#x as string)) as int) AS udf(c1)#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [c1#x]) + +- Project [cast(c1#x as int) AS c1#x] + +- Project [c1#x] + +- SubqueryAlias tab2 + +- LocalRelation [c1#x] + + +-- !query +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1 > udf(6)) +-- !query analysis +Except All true +:- Project [cast(udf(cast(c1#x as string)) as int) AS udf(c1)#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [c1#x]) +: +- Project [cast(c1#x as int) AS c1#x] +: +- Project [c1#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [c1#x] ++- Project [cast(udf(cast(c1#x as string)) as int) AS udf(c1)#x] + +- Filter cast(udf(cast((c1#x > cast(udf(cast(6 as string)) as int)) as string)) as boolean) + +- SubqueryAlias tab2 + +- View (`tab2`, [c1#x]) + +- Project [cast(c1#x as int) AS c1#x] + +- Project [c1#x] + +- SubqueryAlias tab2 + +- LocalRelation [c1#x] + + +-- !query +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT CAST(udf(1) AS BIGINT) +-- !query analysis +Except All true +:- Project [cast(udf(c1)#x as bigint) AS udf(c1)#xL] +: +- Project [cast(udf(cast(c1#x as string)) as int) AS udf(c1)#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [c1#x]) +: +- Project [cast(c1#x as int) AS c1#x] +: +- Project [c1#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [c1#x] ++- Project [cast(cast(udf(cast(1 as string)) as int) as bigint) AS CAST(udf(1) AS BIGINT)#xL] + +- OneRowRelation + + +-- !query +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT array(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"ARRAY\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "EXCEPT ALL", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 51, + "fragment" : "SELECT udf(c1) FROM tab1\nEXCEPT ALL\nSELECT array(1)" + } ] +} + + +-- !query +SELECT udf(k), v FROM tab3 +EXCEPT ALL +SELECT k, udf(v) FROM tab4 +-- !query analysis +Except All true +:- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] +: +- SubqueryAlias tab3 +: +- View (`tab3`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, udf(v) FROM tab4 +EXCEPT ALL +SELECT udf(k), v FROM tab3 +-- !query analysis +Except All true +:- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- SubqueryAlias tab4 +: +- View (`tab4`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab4 +: +- LocalRelation [k#x, v#x] ++- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] + +- SubqueryAlias tab3 + +- View (`tab3`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab3 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(k), udf(v) FROM tab4 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab3 +INTERSECT DISTINCT +SELECT udf(k), udf(v) FROM tab4 +-- !query analysis +Except All true +:- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- SubqueryAlias tab4 +: +- View (`tab4`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab4 +: +- LocalRelation [k#x, v#x] ++- Intersect false + :- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] + : +- SubqueryAlias tab3 + : +- View (`tab3`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias tab3 + : +- LocalRelation [k#x, v#x] + +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(k), v FROM tab4 +EXCEPT ALL +SELECT k, udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), udf(v) FROM tab4 +-- !query analysis +Except false +:- Except All true +: :- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] +: : +- SubqueryAlias tab4 +: : +- View (`tab4`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab4 +: : +- LocalRelation [k#x, v#x] +: +- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- SubqueryAlias tab3 +: +- View (`tab3`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- LocalRelation [k#x, v#x] ++- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab4 +UNION ALL +SELECT udf(k), v FROM tab3 +EXCEPT DISTINCT +SELECT k, udf(v) FROM tab4 +-- !query analysis +Except false +:- Union false, false +: :- Except All true +: : :- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: : : +- SubqueryAlias tab3 +: : : +- View (`tab3`, [k#x,v#x]) +: : : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : : +- Project [k#x, v#x] +: : : +- SubqueryAlias tab3 +: : : +- LocalRelation [k#x, v#x] +: : +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: : +- SubqueryAlias tab4 +: : +- View (`tab4`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab4 +: : +- LocalRelation [k#x, v#x] +: +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] +: +- SubqueryAlias tab3 +: +- View (`tab3`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k FROM tab3 +EXCEPT ALL +SELECT k, v FROM tab4 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_COLUMNS_MISMATCH", + "sqlState" : "42826", + "messageParameters" : { + "firstNumColumns" : "1", + "invalidNumColumns" : "2", + "invalidOrdinalNum" : "second", + "operator" : "EXCEPT ALL" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 51, + "fragment" : "SELECT k FROM tab3\nEXCEPT ALL\nSELECT k, v FROM tab4" + } ] +} + + +-- !query +SELECT udf(k), udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab4 +UNION +SELECT udf(k), udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), udf(v) FROM tab4 +-- !query analysis +Except false +:- Distinct +: +- Union false, false +: :- Except All true +: : :- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: : : +- SubqueryAlias tab3 +: : : +- View (`tab3`, [k#x,v#x]) +: : : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : : +- Project [k#x, v#x] +: : : +- SubqueryAlias tab3 +: : : +- LocalRelation [k#x, v#x] +: : +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: : +- SubqueryAlias tab4 +: : +- View (`tab4`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab4 +: : +- LocalRelation [k#x, v#x] +: +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- SubqueryAlias tab3 +: +- View (`tab3`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- LocalRelation [k#x, v#x] ++- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(k), udf(v) FROM tab3 +MINUS ALL +SELECT k, udf(v) FROM tab4 +UNION +SELECT udf(k), udf(v) FROM tab3 +MINUS DISTINCT +SELECT k, udf(v) FROM tab4 +-- !query analysis +Except false +:- Distinct +: +- Union false, false +: :- Except All true +: : :- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: : : +- SubqueryAlias tab3 +: : : +- View (`tab3`, [k#x,v#x]) +: : : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : : +- Project [k#x, v#x] +: : : +- SubqueryAlias tab3 +: : : +- LocalRelation [k#x, v#x] +: : +- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: : +- SubqueryAlias tab4 +: : +- View (`tab4`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab4 +: : +- LocalRelation [k#x, v#x] +: +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- SubqueryAlias tab3 +: +- View (`tab3`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), v FROM tab4 +EXCEPT DISTINCT +SELECT k, udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), v FROM tab4 +-- !query analysis +Except false +:- Except false +: :- Except All true +: : :- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: : : +- SubqueryAlias tab3 +: : : +- View (`tab3`, [k#x,v#x]) +: : : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : : +- Project [k#x, v#x] +: : : +- SubqueryAlias tab3 +: : : +- LocalRelation [k#x, v#x] +: : +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] +: : +- SubqueryAlias tab4 +: : +- View (`tab4`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab4 +: : +- LocalRelation [k#x, v#x] +: +- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- SubqueryAlias tab3 +: +- View (`tab3`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- LocalRelation [k#x, v#x] ++- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * +FROM (SELECT tab3.k, + udf(tab4.v) + FROM tab3 + JOIN tab4 + ON udf(tab3.k) = tab4.k) +EXCEPT ALL +SELECT * +FROM (SELECT udf(tab3.k), + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = udf(tab4.k)) +-- !query analysis +Except All true +:- Project [k#x, udf(v)#x] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- Join Inner, (cast(udf(cast(k#x as string)) as int) = k#x) +: :- SubqueryAlias tab3 +: : +- View (`tab3`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab3 +: : +- LocalRelation [k#x, v#x] +: +- SubqueryAlias tab4 +: +- View (`tab4`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab4 +: +- LocalRelation [k#x, v#x] ++- Project [udf(k)#x, v#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] + +- Join Inner, (k#x = cast(udf(cast(k#x as string)) as int)) + :- SubqueryAlias tab3 + : +- View (`tab3`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias tab3 + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * +FROM (SELECT udf(udf(tab3.k)), + udf(tab4.v) + FROM tab3 + JOIN tab4 + ON udf(udf(tab3.k)) = udf(tab4.k)) +EXCEPT ALL +SELECT * +FROM (SELECT udf(tab4.v) AS k, + udf(udf(tab3.k)) AS v + FROM tab3 + JOIN tab4 + ON udf(tab3.k) = udf(tab4.k)) +-- !query analysis +Except All true +:- Project [udf(udf(k))#x, udf(v)#x] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int) AS udf(udf(k))#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- Join Inner, (cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int) = cast(udf(cast(k#x as string)) as int)) +: :- SubqueryAlias tab3 +: : +- View (`tab3`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab3 +: : +- LocalRelation [k#x, v#x] +: +- SubqueryAlias tab4 +: +- View (`tab4`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab4 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [cast(udf(cast(v#x as string)) as int) AS k#x, cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int) AS v#x] + +- Join Inner, (cast(udf(cast(k#x as string)) as int) = cast(udf(cast(k#x as string)) as int)) + :- SubqueryAlias tab3 + : +- View (`tab3`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias tab3 + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(v) FROM tab3 GROUP BY v +EXCEPT ALL +SELECT udf(k) FROM tab4 GROUP BY k +-- !query analysis +Except All true +:- Aggregate [v#x], [cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- SubqueryAlias tab3 +: +- View (`tab3`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab3 +: +- LocalRelation [k#x, v#x] ++- Aggregate [k#x], [cast(udf(cast(k#x as string)) as int) AS udf(k)#x] + +- SubqueryAlias tab4 + +- View (`tab4`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab4 + +- LocalRelation [k#x, v#x] + + +-- !query +DROP VIEW IF EXISTS tab1 +-- !query analysis +DropTempViewCommand tab1 + + +-- !query +DROP VIEW IF EXISTS tab2 +-- !query analysis +DropTempViewCommand tab2 + + +-- !query +DROP VIEW IF EXISTS tab3 +-- !query analysis +DropTempViewCommand tab3 + + +-- !query +DROP VIEW IF EXISTS tab4 +-- !query analysis +DropTempViewCommand tab4 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-except.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-except.sql.out new file mode 100644 index 0000000000000..4d3c7323b83b9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-except.sql.out @@ -0,0 +1,247 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", NULL) + as t1(k, v) +-- !query analysis +CreateViewCommand `t1`, select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", NULL) + as t1(k, v), false, false, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias t1 + +- LocalRelation [k#x, v#x] + + +-- !query +create temporary view t2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5), + ("one", NULL), + (NULL, 5) + as t2(k, v) +-- !query analysis +CreateViewCommand `t2`, select * from values + ("one", 1), + ("two", 22), + ("one", 5), + ("one", NULL), + (NULL, 5) + as t2(k, v), false, false, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias t2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(k), udf(v) FROM t1 EXCEPT SELECT udf(k), udf(v) FROM t2 +-- !query analysis +Except false +:- Project [cast(udf(cast(k#x as string)) as string) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [k#x,v#x]) +: +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias t1 +: +- LocalRelation [k#x, v#x] ++- Project [cast(udf(cast(k#x as string)) as string) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] + +- SubqueryAlias t2 + +- View (`t2`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias t2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM t1 EXCEPT SELECT * FROM t1 where udf(v) <> 1 and v <> udf(2) +-- !query analysis +Except false +:- Project [k#x, v#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [k#x,v#x]) +: +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias t1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- Filter (NOT (cast(udf(cast(v#x as string)) as int) = 1) AND NOT (v#x = cast(udf(cast(2 as string)) as int))) + +- SubqueryAlias t1 + +- View (`t1`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias t1 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM t1 where udf(v) <> 1 and v <> udf(22) EXCEPT SELECT * FROM t1 where udf(v) <> 2 and v >= udf(3) +-- !query analysis +Except false +:- Project [k#x, v#x] +: +- Filter (NOT (cast(udf(cast(v#x as string)) as int) = 1) AND NOT (v#x = cast(udf(cast(22 as string)) as int))) +: +- SubqueryAlias t1 +: +- View (`t1`, [k#x,v#x]) +: +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias t1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- Filter (NOT (cast(udf(cast(v#x as string)) as int) = 2) AND (v#x >= cast(udf(cast(3 as string)) as int))) + +- SubqueryAlias t1 + +- View (`t1`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias t1 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT t1.* FROM t1, t2 where t1.k = t2.k +EXCEPT +SELECT t1.* FROM t1, t2 where t1.k = t2.k and t1.k != udf('one') +-- !query analysis +Except false +:- Project [k#x, v#x] +: +- Filter (k#x = k#x) +: +- Join Inner +: :- SubqueryAlias t1 +: : +- View (`t1`, [k#x,v#x]) +: : +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias t1 +: : +- LocalRelation [k#x, v#x] +: +- SubqueryAlias t2 +: +- View (`t2`, [k#x,v#x]) +: +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias t2 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- Filter ((k#x = k#x) AND NOT (k#x = cast(udf(cast(one as string)) as string))) + +- Join Inner + :- SubqueryAlias t1 + : +- View (`t1`, [k#x,v#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias t1 + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias t2 + +- View (`t2`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias t2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM t2 where v >= udf(1) and udf(v) <> 22 EXCEPT SELECT * FROM t1 +-- !query analysis +Except false +:- Project [k#x, v#x] +: +- Filter ((v#x >= cast(udf(cast(1 as string)) as int)) AND NOT (cast(udf(cast(v#x as string)) as int) = 22)) +: +- SubqueryAlias t2 +: +- View (`t2`, [k#x,v#x]) +: +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias t2 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias t1 + +- View (`t1`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias t1 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT (SELECT min(udf(k)) FROM t2 WHERE t2.k = t1.k) min_t2 FROM t1 +MINUS +SELECT (SELECT udf(min(k)) FROM t2) abs_min_t2 FROM t1 WHERE t1.k = udf('one') +-- !query analysis +Except false +:- Project [scalar-subquery#x [k#x] AS min_t2#x] +: : +- Aggregate [min(cast(udf(cast(k#x as string)) as string)) AS min(udf(k))#x] +: : +- Filter (k#x = outer(k#x)) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [k#x,v#x]) +: : +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [k#x, v#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [k#x,v#x]) +: +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias t1 +: +- LocalRelation [k#x, v#x] ++- Project [scalar-subquery#x [] AS abs_min_t2#x] + : +- Aggregate [cast(udf(cast(min(k#x) as string)) as string) AS udf(min(k))#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [k#x,v#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias t2 + : +- LocalRelation [k#x, v#x] + +- Filter (k#x = cast(udf(cast(one as string)) as string)) + +- SubqueryAlias t1 + +- View (`t1`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias t1 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT t1.k +FROM t1 +WHERE t1.v <= (SELECT udf(max(udf(t2.v))) + FROM t2 + WHERE udf(t2.k) = udf(t1.k)) +MINUS +SELECT t1.k +FROM t1 +WHERE udf(t1.v) >= (SELECT min(udf(t2.v)) + FROM t2 + WHERE t2.k = t1.k) +-- !query analysis +Except false +:- Project [k#x] +: +- Filter (v#x <= scalar-subquery#x [k#x]) +: : +- Aggregate [cast(udf(cast(max(cast(udf(cast(v#x as string)) as int)) as string)) as int) AS udf(max(udf(v)))#x] +: : +- Filter (cast(udf(cast(k#x as string)) as string) = cast(udf(cast(outer(k#x) as string)) as string)) +: : +- SubqueryAlias t2 +: : +- View (`t2`, [k#x,v#x]) +: : +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias t2 +: : +- LocalRelation [k#x, v#x] +: +- SubqueryAlias t1 +: +- View (`t1`, [k#x,v#x]) +: +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias t1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x] + +- Filter (cast(udf(cast(v#x as string)) as int) >= scalar-subquery#x [k#x]) + : +- Aggregate [min(cast(udf(cast(v#x as string)) as int)) AS min(udf(v))#x] + : +- Filter (k#x = outer(k#x)) + : +- SubqueryAlias t2 + : +- View (`t2`, [k#x,v#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias t2 + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias t1 + +- View (`t1`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias t1 + +- LocalRelation [k#x, v#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-analytics.sql.out new file mode 100644 index 0000000000000..6c7898722b898 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-analytics.sql.out @@ -0,0 +1,409 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b) +-- !query analysis +CreateViewCommand `testData`, SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b), false, true, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT udf(a + b), b, udf(SUM(a - b)) FROM testData GROUP BY udf(a + b), b WITH CUBE +-- !query analysis +Aggregate [cast(udf(cast((a#x + b#x) as string)) as int)#x, b#x, spark_grouping_id#xL], [cast(udf(cast((a#x + b#x) as string)) as int)#x AS udf((a + b))#x, b#x, cast(udf(cast(sum((a#x - b#x)) as string)) as bigint) AS udf(sum((a - b)))#xL] ++- Expand [[a#x, b#x, cast(udf(cast((a#x + b#x) as string)) as int)#x, b#x, 0], [a#x, b#x, cast(udf(cast((a#x + b#x) as string)) as int)#x, null, 1], [a#x, b#x, null, b#x, 2], [a#x, b#x, null, null, 3]], [a#x, b#x, cast(udf(cast((a#x + b#x) as string)) as int)#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, cast(udf(cast((a#x + b#x) as string)) as int) AS cast(udf(cast((a#x + b#x) as string)) as int)#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT udf(a), udf(b), SUM(b) FROM testData GROUP BY udf(a), b WITH CUBE +-- !query analysis +Aggregate [cast(udf(cast(a#x as string)) as int)#x, b#x, spark_grouping_id#xL], [cast(udf(cast(a#x as string)) as int)#x AS udf(a)#x, cast(udf(cast(b#x as string)) as int) AS udf(b)#x, sum(b#x) AS sum(b)#xL] ++- Expand [[a#x, b#x, cast(udf(cast(a#x as string)) as int)#x, b#x, 0], [a#x, b#x, cast(udf(cast(a#x as string)) as int)#x, null, 1], [a#x, b#x, null, b#x, 2], [a#x, b#x, null, null, 3]], [a#x, b#x, cast(udf(cast(a#x as string)) as int)#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, cast(udf(cast(a#x as string)) as int) AS cast(udf(cast(a#x as string)) as int)#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT udf(a + b), b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP +-- !query analysis +Aggregate [(a#x + b#x)#x, b#x, spark_grouping_id#xL], [cast(udf(cast((a#x + b#x)#x as string)) as int) AS udf((a + b))#x, b#x, sum((a#x - b#x)) AS sum((a - b))#xL] ++- Expand [[a#x, b#x, (a#x + b#x)#x, b#x, 0], [a#x, b#x, (a#x + b#x)#x, null, 1], [a#x, b#x, null, null, 3]], [a#x, b#x, (a#x + b#x)#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, (a#x + b#x) AS (a#x + b#x)#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT udf(a), b, udf(SUM(b)) FROM testData GROUP BY udf(a), b WITH ROLLUP +-- !query analysis +Aggregate [cast(udf(cast(a#x as string)) as int)#x, b#x, spark_grouping_id#xL], [cast(udf(cast(a#x as string)) as int)#x AS udf(a)#x, b#x, cast(udf(cast(sum(b#x) as string)) as bigint) AS udf(sum(b))#xL] ++- Expand [[a#x, b#x, cast(udf(cast(a#x as string)) as int)#x, b#x, 0], [a#x, b#x, cast(udf(cast(a#x as string)) as int)#x, null, 1], [a#x, b#x, null, null, 3]], [a#x, b#x, cast(udf(cast(a#x as string)) as int)#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, cast(udf(cast(a#x as string)) as int) AS cast(udf(cast(a#x as string)) as int)#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW courseSales AS SELECT * FROM VALUES +("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000) +AS courseSales(course, year, earnings) +-- !query analysis +CreateViewCommand `courseSales`, SELECT * FROM VALUES +("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000) +AS courseSales(course, year, earnings), false, true, LocalTempView, true + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY udf(course), year +-- !query analysis +Sort [cast(udf(cast(course#x as string)) as string) ASC NULLS FIRST, year#x ASC NULLS FIRST], true ++- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, sum(earnings#x) AS sum(earnings)#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, udf(year) +-- !query analysis +Sort [course#x ASC NULLS FIRST, cast(udf(cast(year#x as string)) as int) ASC NULLS FIRST], true ++- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, sum(earnings#x) AS sum(earnings)#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, udf(year), SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year) +-- !query analysis +Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, cast(udf(cast(year#x as string)) as int) AS udf(year)#x, sum(earnings#x) AS sum(earnings)#xL] ++- Expand [[course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, udf(SUM(earnings)) FROM courseSales GROUP BY course, year GROUPING SETS(course) +-- !query analysis +Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, cast(udf(cast(sum(earnings#x) as string)) as bigint) AS udf(sum(earnings))#xL] ++- Expand [[course#x, year#x, earnings#x, course#x, null, 1]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT udf(course), year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year) +-- !query analysis +Aggregate [course#x, year#x, spark_grouping_id#xL], [cast(udf(cast(course#x as string)) as string) AS udf(course)#x, year#x, sum(earnings#x) AS sum(earnings)#xL] ++- Expand [[course#x, year#x, earnings#x, null, year#x, 2]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, udf(SUM(earnings)) AS sum FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, udf(sum) +-- !query analysis +Sort [course#x ASC NULLS FIRST, cast(udf(cast(sum#xL as string)) as bigint) ASC NULLS FIRST], true ++- Aggregate [course#x, earnings#x, spark_grouping_id#xL], [course#x, cast(udf(cast(sum(earnings#x) as string)) as bigint) AS sum#xL] + +- Expand [[course#x, year#x, earnings#x, null, null, 3], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, course#x, earnings#x, 0]], [course#x, year#x, earnings#x, course#x, earnings#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, earnings#x AS earnings#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, SUM(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY udf(course), sum +-- !query analysis +Sort [cast(udf(cast(course#x as string)) as string) ASC NULLS FIRST, sum#xL ASC NULLS FIRST], true ++- Aggregate [course#x, earnings#x, spark_grouping_id#xL], [course#x, sum(earnings#x) AS sum#xL, spark_grouping_id#xL AS grouping_id(course, earnings)#xL] + +- Expand [[course#x, year#x, earnings#x, null, null, 3], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, course#x, earnings#x, 0]], [course#x, year#x, earnings#x, course#x, earnings#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, earnings#x AS earnings#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT udf(course), udf(year), GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales +GROUP BY CUBE(course, year) +-- !query analysis +Aggregate [course#x, year#x, spark_grouping_id#xL], [cast(udf(cast(course#x as string)) as string) AS udf(course)#x, cast(udf(cast(year#x as string)) as int) AS udf(year)#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(course)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(year)#x, spark_grouping_id#xL AS grouping_id(course, year)#xL] ++- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, udf(year), GROUPING(course) FROM courseSales GROUP BY course, udf(year) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2445", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 42, + "fragment" : "GROUPING(course)" + } ] +} + + +-- !query +SELECT course, udf(year), GROUPING_ID(course, year) FROM courseSales GROUP BY udf(course), year +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2407", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 51, + "fragment" : "GROUPING_ID(course, year)" + } ] +} + + +-- !query +SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, udf(year) +-- !query analysis +Sort [grouping__id#xL ASC NULLS FIRST, course#x ASC NULLS FIRST, cast(udf(cast(year#x as string)) as int) ASC NULLS FIRST], true ++- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL AS grouping__id#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) +HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, udf(year) +-- !query analysis +Sort [course#x ASC NULLS FIRST, cast(udf(cast(year#x as string)) as int) ASC NULLS FIRST], true ++- Project [course#x, year#x] + +- Filter ((cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) = 1) AND (spark_grouping_id#xL > cast(0 as bigint))) + +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL, spark_grouping_id#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, udf(year) FROM courseSales GROUP BY udf(course), year HAVING GROUPING(course) > 0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION" +} + + +-- !query +SELECT course, udf(udf(year)) FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION" +} + + +-- !query +SELECT udf(course), year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0 +-- !query analysis +Project [udf(course)#x, year#x] ++- Filter (spark_grouping_id#xL > cast(0 as bigint)) + +- Aggregate [course#x, year#x, spark_grouping_id#xL], [cast(udf(cast(course#x as string)) as string) AS udf(course)#x, year#x, spark_grouping_id#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, udf(year) +-- !query analysis +Project [course#x, year#x, grouping(course)#x, grouping(year)#x] ++- Sort [cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) ASC NULLS FIRST, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) ASC NULLS FIRST, course#x ASC NULLS FIRST, cast(udf(cast(year#x as string)) as int) ASC NULLS FIRST], true + +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(course)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(year)#x, spark_grouping_id#xL, spark_grouping_id#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, udf(year) +-- !query analysis +Project [course#x, year#x, grouping_id(course, year)#xL] ++- Sort [cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) ASC NULLS FIRST, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) ASC NULLS FIRST, course#x ASC NULLS FIRST, cast(udf(cast(year#x as string)) as int) ASC NULLS FIRST], true + +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL AS grouping_id(course, year)#xL, spark_grouping_id#xL, spark_grouping_id#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING(course) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION" +} + + +-- !query +SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING_ID(course) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION" +} + + +-- !query +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, udf(course), year +-- !query analysis +Project [course#x, year#x] ++- Sort [spark_grouping_id#xL ASC NULLS FIRST, cast(udf(cast(course#x as string)) as string) ASC NULLS FIRST, year#x ASC NULLS FIRST], true + +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL] + +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] + +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT udf(a + b) AS k1, udf(b) AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2) +-- !query analysis +Aggregate [k1#x, k2#x, spark_grouping_id#xL], [k1#x, k2#x, sum((a#x - b#x)) AS sum((a - b))#xL] ++- Expand [[a#x, b#x, k1#x, k2#x, 0], [a#x, b#x, k1#x, null, 1], [a#x, b#x, null, k2#x, 2], [a#x, b#x, null, null, 3]], [a#x, b#x, k1#x, k2#x, spark_grouping_id#xL] + +- Project [a#x, b#x, cast(udf(cast((a#x + b#x) as string)) as int) AS k1#x, cast(udf(cast(b#x as string)) as int) AS k2#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT udf(udf(a + b)) AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b) +-- !query analysis +Aggregate [k#x, b#x, spark_grouping_id#xL], [k#x, b#x, sum((a#x - b#x)) AS sum((a - b))#xL] ++- Expand [[a#x, b#x, k#x, b#x, 0], [a#x, b#x, k#x, null, 1], [a#x, b#x, null, null, 3]], [a#x, b#x, k#x, b#x, spark_grouping_id#xL] + +- Project [a#x, b#x, cast(udf(cast(cast(udf(cast((a#x + b#x) as string)) as int) as string)) as int) AS k#x, b#x AS b#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT udf(a + b), udf(udf(b)) AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k) +-- !query analysis +Aggregate [(a#x + b#x)#x, k#x, spark_grouping_id#xL], [cast(udf(cast((a#x + b#x)#x as string)) as int) AS udf((a + b))#x, k#x, sum((a#x - b#x)) AS sum((a - b))#xL] ++- Expand [[a#x, b#x, null, k#x, 2]], [a#x, b#x, (a#x + b#x)#x, k#x, spark_grouping_id#xL] + +- Project [a#x, b#x, (a#x + b#x) AS (a#x + b#x)#x, cast(udf(cast(cast(udf(cast(b#x as string)) as int) as string)) as int) AS k#x] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#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 new file mode 100644 index 0000000000000..4b0a46c10dc23 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-by.sql.out @@ -0,0 +1,791 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query analysis +CreateViewCommand `testData`, SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b), false, true, LocalTempView, true + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT udf(a), udf(COUNT(b)) FROM testData +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 42, + "fragment" : "SELECT udf(a), udf(COUNT(b)) FROM testData" + } ] +} + + +-- !query +SELECT COUNT(udf(a)), udf(COUNT(b)) FROM testData +-- !query analysis +Aggregate [count(cast(udf(cast(a#x as string)) as int)) AS count(udf(a))#xL, cast(udf(cast(count(b#x) as string)) as bigint) AS udf(count(b))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT udf(a), COUNT(udf(b)) FROM testData GROUP BY a +-- !query analysis +Aggregate [a#x], [cast(udf(cast(a#x as string)) as int) AS udf(a)#x, count(cast(udf(cast(b#x as string)) as int)) AS count(udf(b))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT COUNT(udf(a)), COUNT(udf(b)) FROM testData GROUP BY udf(a) +-- !query analysis +Aggregate [cast(udf(cast(a#x as string)) as int)], [count(cast(udf(cast(a#x as string)) as int)) AS count(udf(a))#xL, count(cast(udf(cast(b#x as string)) as int)) AS count(udf(b))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1 +-- !query analysis +Aggregate [foo], [foo AS foo#x, count(cast(udf(cast(a#x as string)) as int)) AS count(udf(a))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query analysis +Aggregate [cast(udf(cast(1 as string)) as int)], [foo AS foo#x] ++- Filter (a#x = 0) + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query analysis +Aggregate [cast(udf(cast(1 as string)) as int)], [foo AS foo#x, cast(udf(cast(approx_count_distinct(cast(udf(cast(a#x as string)) as int), 0.05, 0, 0) as string)) as bigint) AS udf(approx_count_distinct(udf(a)))#xL] ++- Filter (a#x = 0) + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query analysis +Aggregate [cast(udf(cast(1 as string)) as int)], [foo AS foo#x, max(struct(col1, cast(udf(cast(a#x as string)) as int))) AS max(struct(udf(a)))#x] ++- Filter (a#x = 0) + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b +-- !query analysis +Aggregate [(a#x + b#x)], [cast(udf(cast((a#x + b#x) as string)) as int) AS udf((a + b))#x, cast(udf(cast(count(b#x) as string)) as bigint) AS udf(count(b))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT udf(a + 1) + 1, udf(COUNT(b)) FROM testData GROUP BY udf(a + 1) +-- !query analysis +Aggregate [cast(udf(cast((a#x + 1) as string)) as int)], [(cast(udf(cast((a#x + 1) as string)) as int) + 1) AS (udf((a + 1)) + 1)#x, cast(udf(cast(count(b#x) as string)) as bigint) AS udf(count(b))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) +FROM testData +-- !query analysis +Aggregate [skewness(cast(cast(udf(cast(a#x as string)) as int) as double)) AS skewness(udf(a))#x, cast(udf(cast(kurtosis(cast(a#x as double)) as string)) as double) AS udf(kurtosis(a))#x, cast(udf(cast(min(a#x) as string)) as int) AS udf(min(a))#x, max(cast(udf(cast(a#x as string)) as int)) AS max(udf(a))#x, cast(udf(cast(avg(cast(udf(cast(a#x as string)) as int)) as string)) as double) AS udf(avg(udf(a)))#x, cast(udf(cast(variance(cast(a#x as double)) as string)) as double) AS udf(variance(a))#x, stddev(cast(cast(udf(cast(a#x as string)) as int) as double)) AS stddev(udf(a))#x, cast(udf(cast(sum(a#x) as string)) as bigint) AS udf(sum(a))#xL, cast(udf(cast(count(a#x) as string)) as bigint) AS udf(count(a))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY udf(a) +-- !query analysis +Aggregate [cast(udf(cast(a#x as string)) as int)], [count(distinct cast(udf(cast(b#x as string)) as int)) AS count(DISTINCT udf(b))#xL, cast(udf(cast(count(distinct b#x, c#x) as string)) as bigint) AS udf(count(DISTINCT b, c))#xL] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [1 AS a#x, 2 AS b#x, 3 AS c#x] + +- OneRowRelation + + +-- !query +SELECT udf(a) AS k, COUNT(udf(b)) FROM testData GROUP BY k +-- !query analysis +Aggregate [cast(udf(cast(a#x as string)) as int)], [cast(udf(cast(a#x as string)) as int) AS k#x, count(cast(udf(cast(b#x as string)) as int)) AS count(udf(b))#xL] ++- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1 +-- !query analysis +Filter (k#x > 1) ++- Aggregate [a#x], [a#x AS k#x, cast(udf(cast(count(b#x) as string)) as bigint) AS udf(count(b))#xL] + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "sqlExpr" : "CAST(udf(cast(count(b) as string)) AS BIGINT)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "udf(COUNT(b))" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) +-- !query analysis +CreateViewCommand `testDataHasSameNameWithAlias`, SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v), false, true, LocalTempView, true + +- Project [k#x, a#x, v#x] + +- SubqueryAlias testDataHasSameNameWithAlias + +- LocalRelation [k#x, a#x, v#x] + + +-- !query +SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"k\"", + "expressionAnyValue" : "\"any_value(k)\"" + } +} + + +-- !query +set spark.sql.groupByAliases=false +-- !query analysis +SetCommand (spark.sql.groupByAliases,Some(false)) + + +-- !query +SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`k`", + "proposal" : "`testdata`.`a`, `testdata`.`b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 58, + "stopIndex" : 58, + "fragment" : "k" + } ] +} + + +-- !query +SELECT udf(a), COUNT(udf(1)) FROM testData WHERE false GROUP BY udf(a) +-- !query analysis +Aggregate [cast(udf(cast(a#x as string)) as int)], [cast(udf(cast(a#x as string)) as int) AS udf(a)#x, count(cast(udf(cast(1 as string)) as int)) AS count(udf(1))#xL] ++- Filter false + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT udf(COUNT(1)) FROM testData WHERE false +-- !query analysis +Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] ++- Filter false + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t +-- !query analysis +Project [1 AS 1#x] ++- SubqueryAlias t + +- Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] + +- Filter false + +- SubqueryAlias testdata + +- View (`testData`, [a#x,b#x]) + +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] + +- Project [a#x, b#x] + +- SubqueryAlias testData + +- LocalRelation [a#x, b#x] + + +-- !query +SELECT 1 from ( + SELECT 1 AS z, + udf(MIN(a.x)) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z +-- !query analysis +Project [1 AS 1#x] ++- Filter NOT (z#x = z#x) + +- SubqueryAlias b + +- Aggregate [1 AS z#x, cast(udf(cast(min(x#x) as string)) as int) AS udf(min(x))#x] + +- Filter false + +- SubqueryAlias a + +- Project [1 AS x#x] + +- OneRowRelation + + +-- !query +SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) +-- !query analysis +Aggregate [corr(distinct cast(x#x as double), cast(y#x as double)) AS corr(DISTINCT x, y)#x, cast(udf(cast(corr(distinct cast(y#x as double), cast(x#x as double)) as string)) as double) AS udf(corr(DISTINCT y, x))#x, count(1) AS count(1)#xL] ++- SubqueryAlias t + +- Project [col1#x AS x#x, col2#x AS y#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT udf(1) FROM range(10) HAVING true +-- !query analysis +Filter cast(true as boolean) ++- Aggregate [cast(udf(cast(1 as string)) as int) AS udf(1)#x] + +- Range (0, 10, step=1, splits=None) + + +-- !query +SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0 +-- !query analysis +Project [udf(udf(1))#x] ++- Filter (max(id#xL)#xL > cast(0 as bigint)) + +- Aggregate [cast(udf(cast(cast(udf(cast(1 as string)) as int) as string)) as int) AS udf(udf(1))#x, max(id#xL) AS max(id#xL)#xL] + +- Range (0, 10, step=1, splits=None) + + +-- !query +SELECT udf(id) FROM range(10) HAVING id > 0 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "SELECT udf(id) FROM range(10) HAVING id > 0" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v) +-- !query analysis +CreateViewCommand `test_agg`, SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v), false, true, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0 +-- !query analysis +Aggregate [cast(udf(cast(every(v#x) as string)) as boolean) AS udf(every(v))#x, cast(udf(cast(some(v#x) as string)) as boolean) AS udf(some(v))#x, any(v#x) AS any(v)#x] ++- Filter (1 = 0) + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4 +-- !query analysis +Aggregate [cast(udf(cast(every(cast(udf(cast(v#x as string)) as boolean)) as string)) as boolean) AS udf(every(udf(v)))#x, some(v#x) AS some(v)#x, any(v#x) AS any(v)#x] ++- Filter (k#x = 4) + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5 +-- !query analysis +Aggregate [every(v#x) AS every(v)#x, cast(udf(cast(some(v#x) as string)) as boolean) AS udf(some(v))#x, any(v#x) AS any(v)#x] ++- Filter (k#x = 5) + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k) +-- !query analysis +Aggregate [cast(udf(cast(k#x as string)) as int)], [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, every(v#x) AS every(v)#x, cast(udf(cast(some(v#x) as string)) as boolean) AS udf(some(v))#x, any(v#x) AS any(v)#x] ++- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false +-- !query analysis +Filter (every(v)#x = false) ++- Aggregate [k#x], [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, every(v#x) AS every(v)#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL +-- !query analysis +Project [udf(k)#x, udf(every(v))#x] ++- Filter isnull(every(v#x)#x) + +- Aggregate [cast(udf(cast(k#x as string)) as int)], [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(every(v#x) as string)) as boolean) AS udf(every(v))#x, every(v#x) AS every(v#x)#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(k), + udf(Every(v)) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(k) +-- !query analysis +Aggregate [cast(udf(cast(k#x as string)) as int)], [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(every(v#x) as string)) as boolean) AS every#x] ++- Filter ((k#x = 2) AND v#x IN (list#x [])) + : +- Aggregate [any(v#x) AS any(v)#x] + : +- Filter (k#x = 1) + : +- SubqueryAlias test_agg + : +- View (`test_agg`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias test_agg + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(udf(k)), + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(udf(k)) +-- !query analysis +Aggregate [cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int)], [cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int) AS udf(udf(k))#x, every(v#x) AS every#x] ++- Filter ((k#x = 2) AND v#x IN (list#x [])) + : +- Aggregate [every(v#x) AS every(v)#x] + : +- Filter (k#x = 1) + : +- SubqueryAlias test_agg + : +- View (`test_agg`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias test_agg + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT every(udf(1)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"udf(1)\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(udf(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "every(udf(1))" + } ] +} + + +-- !query +SELECT some(udf(1S)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"udf(1)\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"some(udf(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "some(udf(1S))" + } ] +} + + +-- !query +SELECT any(udf(1L)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"udf(1)\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"any(udf(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "any(udf(1L))" + } ] +} + + +-- !query +SELECT udf(every("true")) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"true\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 24, + "fragment" : "every(\"true\")" + } ] +} + + +-- !query +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query analysis +Project [k#x, v#x, every(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [k#x, v#x, every(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, every(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [every(v#x) windowspecdefinition(k#x, v#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS every(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [k#x], [v#x ASC NULLS FIRST] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, udf(udf(v)), some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query analysis +Project [k#x, udf(udf(v))#x, some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [k#x, udf(udf(v))#x, v#x, some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [some(v#x) windowspecdefinition(k#x, v#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [k#x], [v#x ASC NULLS FIRST] + +- Project [k#x, cast(udf(cast(cast(udf(cast(v#x as string)) as boolean) as string)) as boolean) AS udf(udf(v))#x, v#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(udf(k)), v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query analysis +Project [udf(udf(k))#x, v#x, any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Project [udf(udf(k))#x, v#x, k#x, any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [any(v#x) windowspecdefinition(k#x, v#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [k#x], [v#x ASC NULLS FIRST] + +- Project [cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int) AS udf(udf(k))#x, v#x, k#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(count(*)) FROM test_agg HAVING count(*) > 1L +-- !query analysis +Project [udf(count(1))#xL] ++- Filter (count(1)#xL > 1) + +- Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL, count(1) AS count(1)#xL] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true +-- !query analysis +Project [k#x, udf(max(v))#x] ++- Filter (max(v#x)#x = true) + +- Aggregate [k#x], [k#x, cast(udf(cast(max(v#x) as string)) as boolean) AS udf(max(v))#x, max(v#x) AS max(v#x)#x] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L +-- !query analysis +Project [cnt#xL] ++- Filter (cnt#xL > 1) + +- SubqueryAlias __auto_generated_subquery_name + +- Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS cnt#xL] + +- SubqueryAlias test_agg + +- View (`test_agg`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as boolean) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias test_agg + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(count(1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L" + } ] +} + + +-- !query +SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"((count(1) + 1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 59, + "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L" + } ] +} + + +-- !query +SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(((k = 1) OR (k = 2)) OR (((count(1) + 1) > 1) OR (max(k) > 1)))\"", + "expressionList" : "count(1), max(test_agg.k)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-having.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-having.sql.out new file mode 100644 index 0000000000000..d4fede4d671c1 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-having.sql.out @@ -0,0 +1,72 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view hav as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v) +-- !query analysis +CreateViewCommand `hav`, select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v), false, false, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias hav + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(k) AS k, udf(sum(v)) FROM hav GROUP BY k HAVING udf(sum(v)) > 2 +-- !query analysis +Filter (udf(sum(v))#xL > cast(2 as bigint)) ++- Aggregate [k#x], [cast(udf(cast(k#x as string)) as string) AS k#x, cast(udf(cast(sum(v#x) as string)) as bigint) AS udf(sum(v))#xL] + +- SubqueryAlias hav + +- View (`hav`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias hav + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(count(udf(k))) FROM hav GROUP BY v + 1 HAVING v + 1 = udf(2) +-- !query analysis +Project [udf(count(udf(k)))#xL] ++- Filter ((v#x + 1)#x = cast(udf(cast(2 as string)) as int)) + +- Aggregate [(v#x + 1)], [cast(udf(cast(count(cast(udf(cast(k#x as string)) as string)) as string)) as bigint) AS udf(count(udf(k)))#xL, (v#x + 1) AS (v#x + 1)#x] + +- SubqueryAlias hav + +- View (`hav`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias hav + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(MIN(t.v)) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(udf(COUNT(udf(1))) > 0) +-- !query analysis +Project [udf(min(v))#x] ++- Filter (cast(udf(cast(count(cast(udf(cast(1 as string)) as int))#xL as string)) as bigint) > cast(0 as bigint)) + +- Aggregate [cast(udf(cast(min(v#x) as string)) as int) AS udf(min(v))#x, count(cast(udf(cast(1 as string)) as int)) AS count(cast(udf(cast(1 as string)) as int))#xL] + +- SubqueryAlias t + +- Project [k#x, v#x] + +- Filter (v#x > 0) + +- SubqueryAlias hav + +- View (`hav`, [k#x,v#x]) + +- Project [cast(k#x as string) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias hav + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(a + b) FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > udf(1) +-- !query analysis +Project [udf((a + b))#xL] ++- Filter ((a#xL + cast(b#x as bigint))#xL > cast(cast(udf(cast(1 as string)) as int) as bigint)) + +- Aggregate [(a#xL + cast(b#x as bigint))], [cast(udf(cast((a#xL + cast(b#x as bigint)) as string)) as bigint) AS udf((a + b))#xL, (a#xL + cast(b#x as bigint)) AS (a#xL + cast(b#x as bigint))#xL] + +- SubqueryAlias T + +- LocalRelation [a#xL, b#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-inline-table.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-inline-table.sql.out new file mode 100644 index 0000000000000..7c9ace74676f4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-inline-table.sql.out @@ -0,0 +1,206 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select udf(col1), udf(col2) from values ("one", 1) +-- !query analysis +Project [cast(udf(cast(col1#x as string)) as string) AS udf(col1)#x, cast(udf(cast(col2#x as string)) as int) AS udf(col2)#x] ++- LocalRelation [col1#x, col2#x] + + +-- !query +select udf(col1), udf(udf(col2)) from values ("one", 1) as data +-- !query analysis +Project [cast(udf(cast(col1#x as string)) as string) AS udf(col1)#x, cast(udf(cast(cast(udf(cast(col2#x as string)) as int) as string)) as int) AS udf(udf(col2))#x] ++- SubqueryAlias data + +- LocalRelation [col1#x, col2#x] + + +-- !query +select udf(a), b from values ("one", 1) as data(a, b) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as string) AS udf(a)#x, b#x] ++- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select udf(a) from values 1, 2, 3 as data(a) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as int) AS udf(a)#x] ++- SubqueryAlias data + +- LocalRelation [a#x] + + +-- !query +select udf(a), b from values ("one", 1), ("two", 2), ("three", null) as data(a, b) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as string) AS udf(a)#x, b#x] ++- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select udf(a), b from values ("one", null), ("two", null) as data(a, b) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as string) AS udf(a)#x, b#x] ++- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select udf(a), b from values ("one", 1), ("two", 2L) as data(a, b) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as string) AS udf(a)#x, b#xL] ++- SubqueryAlias data + +- LocalRelation [a#x, b#xL] + + +-- !query +select udf(udf(a)), udf(b) from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b) +-- !query analysis +Project [cast(udf(cast(cast(udf(cast(a#x as string)) as string) as string)) as string) AS udf(udf(a))#x, cast(udf(cast(b#xL as string)) as bigint) AS udf(b)#xL] ++- SubqueryAlias data + +- LocalRelation [a#x, b#xL] + + +-- !query +select udf(a), b from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as string) AS udf(a)#x, b#x] ++- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select udf(a), b from values ("one", 2.0), ("two", 3.0D) as data(a, b) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as string) AS udf(a)#x, b#x] ++- SubqueryAlias data + +- LocalRelation [a#x, b#x] + + +-- !query +select udf(a), b from values ("one", rand(5)), ("two", 3.0D) as data(a, b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2304", + "messageParameters" : { + "sqlExpr" : "rand(5)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 38, + "stopIndex" : 44, + "fragment" : "rand(5)" + } ] +} + + +-- !query +select udf(a), udf(b) from values ("one", 2.0), ("two") as data(a, b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2305", + "messageParameters" : { + "numCols" : "2", + "ri" : "1", + "rowSize" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 69, + "fragment" : "values (\"one\", 2.0), (\"two\") as data(a, b)" + } ] +} + + +-- !query +select udf(a), udf(b) from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2303", + "messageParameters" : { + "name" : "b" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 91, + "fragment" : "values (\"one\", array(0, 1)), (\"two\", struct(1, 2)) as data(a, b)" + } ] +} + + +-- !query +select udf(a), udf(b) from values ("one"), ("two") as data(a, b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2305", + "messageParameters" : { + "numCols" : "2", + "ri" : "0", + "rowSize" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 64, + "fragment" : "values (\"one\"), (\"two\") as data(a, b)" + } ] +} + + +-- !query +select udf(a), udf(b) from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`random_not_exist_func`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 43, + "stopIndex" : 66, + "fragment" : "random_not_exist_func(1)" + } ] +} + + +-- !query +select udf(a), udf(b) from values ("one", count(1)), ("two", 2) as data(a, b) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2304", + "messageParameters" : { + "sqlExpr" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 43, + "stopIndex" : 50, + "fragment" : "count(1)" + } ] +} + + +-- !query +select udf(a), b from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as timestamp) AS udf(a)#x, b#x] ++- SubqueryAlias data + +- LocalRelation [a#x, b#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-inner-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-inner-join.sql.out new file mode 100644 index 0000000000000..716098ca78484 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-inner-join.sql.out @@ -0,0 +1,130 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query analysis +CreateViewCommand `t1`, SELECT * FROM VALUES (1) AS GROUPING(a), false, false, LocalTempView, true + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query analysis +CreateViewCommand `t2`, SELECT * FROM VALUES (1) AS GROUPING(a), false, false, LocalTempView, true + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) +-- !query analysis +CreateViewCommand `t3`, SELECT * FROM VALUES (1), (1) AS GROUPING(a), false, false, LocalTempView, true + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) +-- !query analysis +CreateViewCommand `t4`, SELECT * FROM VALUES (1), (1) AS GROUPING(a), false, false, LocalTempView, true + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +CREATE TEMPORARY VIEW ta AS +SELECT udf(a) AS a, udf('a') AS tag FROM t1 +UNION ALL +SELECT udf(a) AS a, udf('b') AS tag FROM t2 +-- !query analysis +CreateViewCommand `ta`, SELECT udf(a) AS a, udf('a') AS tag FROM t1 +UNION ALL +SELECT udf(a) AS a, udf('b') AS tag FROM t2, false, false, LocalTempView, true, [udf] + +- Union false, false + :- Project [cast(udf(cast(a#x as string)) as int) AS a#x, cast(udf(cast(a as string)) as string) AS tag#x] + : +- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- Project [cast(udf(cast(a#x as string)) as int) AS a#x, cast(udf(cast(b as string)) as string) AS tag#x] + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +CREATE TEMPORARY VIEW tb AS +SELECT udf(a) AS a, udf('a') AS tag FROM t3 +UNION ALL +SELECT udf(a) AS a, udf('b') AS tag FROM t4 +-- !query analysis +CreateViewCommand `tb`, SELECT udf(a) AS a, udf('a') AS tag FROM t3 +UNION ALL +SELECT udf(a) AS a, udf('b') AS tag FROM t4, false, false, LocalTempView, true, [udf] + +- Union false, false + :- Project [cast(udf(cast(a#x as string)) as int) AS a#x, cast(udf(cast(a as string)) as string) AS tag#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- Project [cast(udf(cast(a#x as string)) as int) AS a#x, cast(udf(cast(b as string)) as string) AS tag#x] + +- SubqueryAlias t4 + +- View (`t4`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag +-- !query analysis +Project [a#x, tag#x] ++- Join Inner, ((a#x = a#x) AND (tag#x = tag#x)) + :- SubqueryAlias ta + : +- View (`ta`, [a#x,tag#x]) + : +- Project [cast(a#x as int) AS a#x, cast(tag#x as string) AS tag#x] + : +- Union false, false + : :- Project [cast(udf(cast(a#x as string)) as int) AS a#x, cast(udf(cast(a as string)) as string) AS tag#x] + : : +- SubqueryAlias t1 + : : +- View (`t1`, [a#x]) + : : +- Project [cast(a#x as int) AS a#x] + : : +- Project [a#x] + : : +- SubqueryAlias GROUPING + : : +- LocalRelation [a#x] + : +- Project [cast(udf(cast(a#x as string)) as int) AS a#x, cast(udf(cast(b as string)) as string) AS tag#x] + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias tb + +- View (`tb`, [a#x,tag#x]) + +- Project [cast(a#x as int) AS a#x, cast(tag#x as string) AS tag#x] + +- Union false, false + :- Project [cast(udf(cast(a#x as string)) as int) AS a#x, cast(udf(cast(a as string)) as string) AS tag#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- Project [cast(udf(cast(a#x as string)) as int) AS a#x, cast(udf(cast(b as string)) as string) AS tag#x] + +- SubqueryAlias t4 + +- View (`t4`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-intersect-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-intersect-all.sql.out new file mode 100644 index 0000000000000..ad026de10ed1a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-intersect-all.sql.out @@ -0,0 +1,602 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (1, 3), + (2, 3), + (null, null), + (null, null) + AS tab1(k, v) +-- !query analysis +CreateViewCommand `tab1`, SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (1, 3), + (2, 3), + (null, null), + (null, null) + AS tab1(k, v), false, false, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias tab1 + +- LocalRelation [k#x, v#x] + + +-- !query +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (2, 3), + (3, 4), + (null, null), + (null, null) + AS tab2(k, v) +-- !query analysis +CreateViewCommand `tab2`, SELECT * FROM VALUES + (1, 2), + (1, 2), + (2, 3), + (3, 4), + (null, null), + (null, null) + AS tab2(k, v), false, false, LocalTempView, true + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT k, udf(v) FROM tab2 +-- !query analysis +Intersect All true +:- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab1 WHERE udf(k) = 1 +-- !query analysis +Intersect All true +:- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] + +- Filter (cast(udf(cast(k#x as string)) as int) = 1) + +- SubqueryAlias tab1 + +- View (`tab1`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab1 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(k), udf(v) FROM tab1 WHERE k > udf(2) +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2 +-- !query analysis +Intersect All true +:- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- Filter (k#x > cast(udf(cast(2 as string)) as int)) +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab2 WHERE udf(udf(k)) > 3 +-- !query analysis +Intersect All true +:- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] + +- Filter (cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int) > 3) + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT CAST(udf(1) AS BIGINT), CAST(udf(2) AS BIGINT) +-- !query analysis +Intersect All true +:- Project [cast(udf(k)#x as bigint) AS udf(k)#xL, cast(v#x as bigint) AS v#xL] +: +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [cast(cast(udf(cast(1 as string)) as int) as bigint) AS CAST(udf(1) AS BIGINT)#xL, cast(cast(udf(cast(2 as string)) as int) as bigint) AS CAST(udf(2) AS BIGINT)#xL] + +- OneRowRelation + + +-- !query +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT array(1), udf(2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"ARRAY\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "INTERSECT ALL", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 64, + "fragment" : "SELECT k, udf(v) FROM tab1\nINTERSECT ALL\nSELECT array(1), udf(2)" + } ] +} + + +-- !query +SELECT udf(k) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_COLUMNS_MISMATCH", + "sqlState" : "42826", + "messageParameters" : { + "firstNumColumns" : "1", + "invalidNumColumns" : "2", + "invalidOrdinalNum" : "second", + "operator" : "INTERSECT ALL" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 69, + "fragment" : "SELECT udf(k) FROM tab1\nINTERSECT ALL\nSELECT udf(k), udf(v) FROM tab2" + } ] +} + + +-- !query +SELECT udf(k), v FROM tab2 +INTERSECT ALL +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2 +-- !query analysis +Intersect All true +:- Intersect All true +: :- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] +: : +- SubqueryAlias tab2 +: : +- View (`tab2`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab2 +: : +- LocalRelation [k#x, v#x] +: +- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(k), v FROM tab1 +EXCEPT +SELECT k, udf(v) FROM tab2 +UNION ALL +SELECT k, udf(udf(v)) FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab2 +-- !query analysis +Union false, false +:- Except false +: :- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] +: : +- SubqueryAlias tab1 +: : +- View (`tab1`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab1 +: : +- LocalRelation [k#x, v#x] +: +- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- SubqueryAlias tab2 +: +- View (`tab2`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab2 +: +- LocalRelation [k#x, v#x] ++- Intersect All true + :- Project [k#x, cast(udf(cast(cast(udf(cast(v#x as string)) as int) as string)) as int) AS udf(udf(v))#x] + : +- SubqueryAlias tab1 + : +- View (`tab1`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias tab1 + : +- LocalRelation [k#x, v#x] + +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(k), udf(v) FROM tab1 +EXCEPT +SELECT udf(k), v FROM tab2 +EXCEPT +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(udf(v)) FROM tab2 +-- !query analysis +Except false +:- Except false +: :- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: : +- SubqueryAlias tab1 +: : +- View (`tab1`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab1 +: : +- LocalRelation [k#x, v#x] +: +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] +: +- SubqueryAlias tab2 +: +- View (`tab2`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab2 +: +- LocalRelation [k#x, v#x] ++- Intersect All true + :- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] + : +- SubqueryAlias tab1 + : +- View (`tab1`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias tab1 + : +- LocalRelation [k#x, v#x] + +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(cast(udf(cast(v#x as string)) as int) as string)) as int) AS udf(udf(v))#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +( + ( + ( + SELECT udf(k), v FROM tab1 + EXCEPT + SELECT k, udf(v) FROM tab2 + ) + EXCEPT + SELECT udf(k), udf(v) FROM tab1 + ) + INTERSECT ALL + SELECT udf(k), udf(v) FROM tab2 +) +-- !query analysis +Intersect All true +:- Except false +: :- Except false +: : :- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] +: : : +- SubqueryAlias tab1 +: : : +- View (`tab1`, [k#x,v#x]) +: : : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : : +- Project [k#x, v#x] +: : : +- SubqueryAlias tab1 +: : : +- LocalRelation [k#x, v#x] +: : +- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: : +- SubqueryAlias tab2 +: : +- View (`tab2`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab2 +: : +- LocalRelation [k#x, v#x] +: +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(udf(tab1.k)) = tab2.k) +INTERSECT ALL +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(tab1.k) = udf(udf(tab2.k))) +-- !query analysis +Intersect All true +:- Project [udf(k)#x, udf(v)#x] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- Join Inner, (cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int) = k#x) +: :- SubqueryAlias tab1 +: : +- View (`tab1`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab1 +: : +- LocalRelation [k#x, v#x] +: +- SubqueryAlias tab2 +: +- View (`tab2`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab2 +: +- LocalRelation [k#x, v#x] ++- Project [udf(k)#x, udf(v)#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] + +- Join Inner, (cast(udf(cast(k#x as string)) as int) = cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int)) + :- SubqueryAlias tab1 + : +- View (`tab1`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias tab1 + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(tab1.k) = udf(tab2.k)) +INTERSECT ALL +SELECT * +FROM (SELECT udf(tab2.v) AS k, + udf(tab1.k) AS v + FROM tab1 + JOIN tab2 + ON tab1.k = udf(tab2.k)) +-- !query analysis +Intersect All true +:- Project [udf(k)#x, udf(v)#x] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- Join Inner, (cast(udf(cast(k#x as string)) as int) = cast(udf(cast(k#x as string)) as int)) +: :- SubqueryAlias tab1 +: : +- View (`tab1`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab1 +: : +- LocalRelation [k#x, v#x] +: +- SubqueryAlias tab2 +: +- View (`tab2`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab2 +: +- LocalRelation [k#x, v#x] ++- Project [k#x, v#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [cast(udf(cast(v#x as string)) as int) AS k#x, cast(udf(cast(k#x as string)) as int) AS v#x] + +- Join Inner, (k#x = cast(udf(cast(k#x as string)) as int)) + :- SubqueryAlias tab1 + : +- View (`tab1`, [k#x,v#x]) + : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + : +- Project [k#x, v#x] + : +- SubqueryAlias tab1 + : +- LocalRelation [k#x, v#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT udf(v) FROM tab1 GROUP BY v +INTERSECT ALL +SELECT udf(udf(k)) FROM tab2 GROUP BY k +-- !query analysis +Intersect All true +:- Aggregate [v#x], [cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Aggregate [k#x], [cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int) AS udf(udf(k))#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SET spark.sql.legacy.setopsPrecedence.enabled= true +-- !query analysis +SetCommand (spark.sql.legacy.setopsPrecedence.enabled,Some(true)) + + +-- !query +SELECT udf(k), v FROM tab1 +EXCEPT +SELECT k, udf(v) FROM tab2 +UNION ALL +SELECT udf(k), udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(udf(k)), udf(v) FROM tab2 +-- !query analysis +Intersect All true +:- Union false, false +: :- Except false +: : :- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] +: : : +- SubqueryAlias tab1 +: : : +- View (`tab1`, [k#x,v#x]) +: : : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : : +- Project [k#x, v#x] +: : : +- SubqueryAlias tab1 +: : : +- LocalRelation [k#x, v#x] +: : +- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: : +- SubqueryAlias tab2 +: : +- View (`tab2`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab2 +: : +- LocalRelation [k#x, v#x] +: +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as int) AS udf(udf(k))#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SELECT k, udf(v) FROM tab1 +EXCEPT +SELECT udf(k), v FROM tab2 +UNION ALL +SELECT udf(k), udf(v) FROM tab1 +INTERSECT +SELECT udf(k), udf(udf(v)) FROM tab2 +-- !query analysis +Intersect false +:- Union false, false +: :- Except false +: : :- Project [k#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: : : +- SubqueryAlias tab1 +: : : +- View (`tab1`, [k#x,v#x]) +: : : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : : +- Project [k#x, v#x] +: : : +- SubqueryAlias tab1 +: : : +- LocalRelation [k#x, v#x] +: : +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, v#x] +: : +- SubqueryAlias tab2 +: : +- View (`tab2`, [k#x,v#x]) +: : +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: : +- Project [k#x, v#x] +: : +- SubqueryAlias tab2 +: : +- LocalRelation [k#x, v#x] +: +- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(v#x as string)) as int) AS udf(v)#x] +: +- SubqueryAlias tab1 +: +- View (`tab1`, [k#x,v#x]) +: +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] +: +- Project [k#x, v#x] +: +- SubqueryAlias tab1 +: +- LocalRelation [k#x, v#x] ++- Project [cast(udf(cast(k#x as string)) as int) AS udf(k)#x, cast(udf(cast(cast(udf(cast(v#x as string)) as int) as string)) as int) AS udf(udf(v))#x] + +- SubqueryAlias tab2 + +- View (`tab2`, [k#x,v#x]) + +- Project [cast(k#x as int) AS k#x, cast(v#x as int) AS v#x] + +- Project [k#x, v#x] + +- SubqueryAlias tab2 + +- LocalRelation [k#x, v#x] + + +-- !query +SET spark.sql.legacy.setopsPrecedence.enabled = false +-- !query analysis +SetCommand (spark.sql.legacy.setopsPrecedence.enabled,Some(false)) + + +-- !query +DROP VIEW IF EXISTS tab1 +-- !query analysis +DropTempViewCommand tab1 + + +-- !query +DROP VIEW IF EXISTS tab2 +-- !query analysis +DropTempViewCommand tab2 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-join-empty-relation.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-join-empty-relation.sql.out new file mode 100644 index 0000000000000..d5ffa3ebb2bb5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-join-empty-relation.sql.out @@ -0,0 +1,577 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query analysis +CreateViewCommand `t1`, SELECT * FROM VALUES (1) AS GROUPING(a), false, false, LocalTempView, true + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query analysis +CreateViewCommand `t2`, SELECT * FROM VALUES (1) AS GROUPING(a), false, false, LocalTempView, true + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +CREATE TEMPORARY VIEW empty_table as SELECT a FROM t2 WHERE false +-- !query analysis +CreateViewCommand `empty_table`, SELECT a FROM t2 WHERE false, false, false, LocalTempView, true + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(t1.a), udf(empty_table.a) FROM t1 INNER JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a))) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as int) AS udf(a)#x, cast(udf(cast(a#x as string)) as int) AS udf(a)#x] ++- Join Inner, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int)) + :- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(t1.a), udf(udf(empty_table.a)) FROM t1 CROSS JOIN empty_table ON (udf(udf(t1.a)) = udf(empty_table.a)) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as int) AS udf(a)#x, cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) AS udf(udf(a))#x] ++- Join Cross, (cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) = cast(udf(cast(a#x as string)) as int)) + :- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(udf(t1.a)), empty_table.a FROM t1 LEFT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query analysis +Project [cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) AS udf(udf(a))#x, a#x] ++- Join LeftOuter, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(a#x as string)) as int)) + :- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(t1.a), udf(empty_table.a) FROM t1 RIGHT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as int) AS udf(a)#x, cast(udf(cast(a#x as string)) as int) AS udf(a)#x] ++- Join RightOuter, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(a#x as string)) as int)) + :- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(t1.a), empty_table.a FROM t1 FULL OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as int) AS udf(a)#x, a#x] ++- Join FullOuter, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(a#x as string)) as int)) + :- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(udf(t1.a)) FROM t1 LEFT SEMI JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a))) +-- !query analysis +Project [cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) AS udf(udf(a))#x] ++- Join LeftSemi, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int)) + :- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(t1.a) FROM t1 LEFT ANTI JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as int) AS udf(a)#x] ++- Join LeftAnti, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(a#x as string)) as int)) + :- SubqueryAlias t1 + : +- View (`t1`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(empty_table.a), udf(t1.a) FROM empty_table INNER JOIN t1 ON (udf(udf(empty_table.a)) = udf(t1.a)) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as int) AS udf(a)#x, cast(udf(cast(a#x as string)) as int) AS udf(a)#x] ++- Join Inner, (cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) = cast(udf(cast(a#x as string)) as int)) + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias t1 + +- View (`t1`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(empty_table.a), udf(udf(t1.a)) FROM empty_table CROSS JOIN t1 ON (udf(empty_table.a) = udf(udf(t1.a))) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as int) AS udf(a)#x, cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) AS udf(udf(a))#x] ++- Join Cross, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int)) + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias t1 + +- View (`t1`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(udf(empty_table.a)), udf(t1.a) FROM empty_table LEFT OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) +-- !query analysis +Project [cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) AS udf(udf(a))#x, cast(udf(cast(a#x as string)) as int) AS udf(a)#x] ++- Join LeftOuter, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(a#x as string)) as int)) + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias t1 + +- View (`t1`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT empty_table.a, udf(t1.a) FROM empty_table RIGHT OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) +-- !query analysis +Project [a#x, cast(udf(cast(a#x as string)) as int) AS udf(a)#x] ++- Join RightOuter, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(a#x as string)) as int)) + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias t1 + +- View (`t1`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT empty_table.a, udf(udf(t1.a)) FROM empty_table FULL OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) +-- !query analysis +Project [a#x, cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) AS udf(udf(a))#x] ++- Join FullOuter, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(a#x as string)) as int)) + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias t1 + +- View (`t1`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(udf(empty_table.a)) FROM empty_table LEFT SEMI JOIN t1 ON (udf(empty_table.a) = udf(udf(t1.a))) +-- !query analysis +Project [cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) AS udf(udf(a))#x] ++- Join LeftSemi, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int)) + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias t1 + +- View (`t1`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT empty_table.a FROM empty_table LEFT ANTI JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) +-- !query analysis +Project [a#x] ++- Join LeftAnti, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(a#x as string)) as int)) + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias t1 + +- View (`t1`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(empty_table.a) FROM empty_table INNER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(udf(empty_table2.a))) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as int) AS udf(a)#x] ++- Join Inner, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int)) + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table2 + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(udf(empty_table.a)) FROM empty_table CROSS JOIN empty_table AS empty_table2 ON (udf(udf(empty_table.a)) = udf(empty_table2.a)) +-- !query analysis +Project [cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) AS udf(udf(a))#x] ++- Join Cross, (cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) = cast(udf(cast(a#x as string)) as int)) + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table2 + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(empty_table.a) FROM empty_table LEFT OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as int) AS udf(a)#x] ++- Join LeftOuter, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(a#x as string)) as int)) + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table2 + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(udf(empty_table.a)) FROM empty_table RIGHT OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(udf(empty_table2.a))) +-- !query analysis +Project [cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) AS udf(udf(a))#x] ++- Join RightOuter, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int)) + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table2 + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(empty_table.a) FROM empty_table FULL OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as int) AS udf(a)#x] ++- Join FullOuter, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(a#x as string)) as int)) + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table2 + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(udf(empty_table.a)) FROM empty_table LEFT SEMI JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) +-- !query analysis +Project [cast(udf(cast(cast(udf(cast(a#x as string)) as int) as string)) as int) AS udf(udf(a))#x] ++- Join LeftSemi, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(a#x as string)) as int)) + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table2 + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] + + +-- !query +SELECT udf(empty_table.a) FROM empty_table LEFT ANTI JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) +-- !query analysis +Project [cast(udf(cast(a#x as string)) as int) AS udf(a)#x] ++- Join LeftAnti, (cast(udf(cast(a#x as string)) as int) = cast(udf(cast(a#x as string)) as int)) + :- SubqueryAlias empty_table + : +- View (`empty_table`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- Filter false + : +- SubqueryAlias t2 + : +- View (`t2`, [a#x]) + : +- Project [cast(a#x as int) AS a#x] + : +- Project [a#x] + : +- SubqueryAlias GROUPING + : +- LocalRelation [a#x] + +- SubqueryAlias empty_table2 + +- SubqueryAlias empty_table + +- View (`empty_table`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- Filter false + +- SubqueryAlias t2 + +- View (`t2`, [a#x]) + +- Project [cast(a#x as int) AS a#x] + +- Project [a#x] + +- SubqueryAlias GROUPING + +- LocalRelation [a#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-natural-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-natural-join.sql.out new file mode 100644 index 0000000000000..e43dc13177f51 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-natural-join.sql.out @@ -0,0 +1,118 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query analysis +CreateViewCommand `nt1`, select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1), false, false, LocalTempView, true + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- LocalRelation [k#x, v1#x] + + +-- !query +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query analysis +CreateViewCommand `nt2`, select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2), false, false, LocalTempView, true + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 natural join nt2 where udf(k) = "one" +-- !query analysis +Project [k#x, v1#x, v2#x] ++- Filter (cast(udf(cast(k#x as string)) as string) = one) + +- Project [k#x, v1#x, v2#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 natural left join nt2 where k <> udf("") order by v1, v2 +-- !query analysis +Sort [v1#x ASC NULLS FIRST, v2#x ASC NULLS FIRST], true ++- Project [k#x, v1#x, v2#x] + +- Filter NOT (k#x = cast(udf(cast( as string)) as string)) + +- Project [k#x, v1#x, v2#x] + +- Join LeftOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 natural right join nt2 where udf(k) <> udf("") order by v1, v2 +-- !query analysis +Sort [v1#x ASC NULLS FIRST, v2#x ASC NULLS FIRST], true ++- Project [k#x, v1#x, v2#x] + +- Filter NOT (cast(udf(cast(k#x as string)) as string) = cast(udf(cast( as string)) as string)) + +- Project [k#x, v1#x, v2#x] + +- Join RightOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT udf(count(*)) FROM nt1 natural full outer join nt2 +-- !query analysis +Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] ++- Project [coalesce(k#x, k#x) AS k#x, v1#x, v2#x] + +- Join FullOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-outer-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-outer-join.sql.out new file mode 100644 index 0000000000000..7334ebbb1b92d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-outer-join.sql.out @@ -0,0 +1,113 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(-234), (145), (367), (975), (298) +as t1(int_col1) +-- !query analysis +CreateViewCommand `t1`, SELECT * FROM VALUES +(-234), (145), (367), (975), (298) +as t1(int_col1), false, true, LocalTempView, true + +- Project [int_col1#x] + +- SubqueryAlias t1 + +- LocalRelation [int_col1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES +(-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) +as t2(int_col0, int_col1) +-- !query analysis +CreateViewCommand `t2`, SELECT * FROM VALUES +(-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) +as t2(int_col0, int_col1), false, true, LocalTempView, true + +- Project [int_col0#x, int_col1#x] + +- SubqueryAlias t2 + +- LocalRelation [int_col0#x, int_col1#x] + + +-- !query +SELECT + (udf(SUM(udf(COALESCE(t1.int_col1, t2.int_col0))))), + (udf(COALESCE(t1.int_col1, t2.int_col0)) * 2) +FROM t1 +RIGHT JOIN t2 + ON udf(t2.int_col0) = udf(t1.int_col1) +GROUP BY udf(GREATEST(COALESCE(udf(t2.int_col1), 109), COALESCE(t1.int_col1, udf(-449)))), + COALESCE(t1.int_col1, t2.int_col0) +HAVING (udf(SUM(COALESCE(udf(t1.int_col1), udf(t2.int_col0))))) + > (udf(COALESCE(t1.int_col1, t2.int_col0)) * 2) +-- !query analysis +Project [udf(sum(udf(coalesce(int_col1, int_col0))))#xL, (udf(coalesce(int_col1, int_col0)) * 2)#x] ++- Filter (cast(udf(cast(sum(coalesce(cast(udf(cast(int_col1#x as string)) as int), cast(udf(cast(int_col0#x as string)) as int)))#xL as string)) as bigint) > cast((udf(coalesce(int_col1, int_col0)) * 2)#x as bigint)) + +- Aggregate [cast(udf(cast(greatest(coalesce(cast(udf(cast(int_col1#x as string)) as int), 109), coalesce(int_col1#x, cast(udf(cast(-449 as string)) as int))) as string)) as int), coalesce(int_col1#x, int_col0#x)], [cast(udf(cast(sum(cast(udf(cast(coalesce(int_col1#x, int_col0#x) as string)) as int)) as string)) as bigint) AS udf(sum(udf(coalesce(int_col1, int_col0))))#xL, (cast(udf(cast(coalesce(int_col1#x, int_col0#x) as string)) as int) * 2) AS (udf(coalesce(int_col1, int_col0)) * 2)#x, sum(coalesce(cast(udf(cast(int_col1#x as string)) as int), cast(udf(cast(int_col0#x as string)) as int))) AS sum(coalesce(cast(udf(cast(int_col1#x as string)) as int), cast(udf(cast(int_col0#x as string)) as int)))#xL] + +- Join RightOuter, (cast(udf(cast(int_col0#x as string)) as int) = cast(udf(cast(int_col1#x as string)) as int)) + :- SubqueryAlias t1 + : +- View (`t1`, [int_col1#x]) + : +- Project [cast(int_col1#x as int) AS int_col1#x] + : +- Project [int_col1#x] + : +- SubqueryAlias t1 + : +- LocalRelation [int_col1#x] + +- SubqueryAlias t2 + +- View (`t2`, [int_col0#x,int_col1#x]) + +- Project [cast(int_col0#x as int) AS int_col0#x, cast(int_col1#x as int) AS int_col1#x] + +- Project [int_col0#x, int_col1#x] + +- SubqueryAlias t2 + +- LocalRelation [int_col0#x, int_col1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1) +-- !query analysis +CreateViewCommand `t1`, SELECT * FROM VALUES (97) as t1(int_col1), false, true, LocalTempView, true + +- Project [int_col1#x] + +- SubqueryAlias t1 + +- LocalRelation [int_col1#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1) +-- !query analysis +CreateViewCommand `t2`, SELECT * FROM VALUES (0) as t2(int_col1), false, true, LocalTempView, true + +- Project [int_col1#x] + +- SubqueryAlias t2 + +- LocalRelation [int_col1#x] + + +-- !query +set spark.sql.crossJoin.enabled = true +-- !query analysis +SetCommand (spark.sql.crossJoin.enabled,Some(true)) + + +-- !query +SELECT * +FROM ( +SELECT + udf(COALESCE(udf(t2.int_col1), udf(t1.int_col1))) AS int_col + FROM t1 + LEFT JOIN t2 ON false +) t where (udf(t.int_col)) is not null +-- !query analysis +Project [int_col#x] ++- Filter isnotnull(cast(udf(cast(int_col#x as string)) as int)) + +- SubqueryAlias t + +- Project [cast(udf(cast(coalesce(cast(udf(cast(int_col1#x as string)) as int), cast(udf(cast(int_col1#x as string)) as int)) as string)) as int) AS int_col#x] + +- Join LeftOuter, false + :- SubqueryAlias t1 + : +- View (`t1`, [int_col1#x]) + : +- Project [cast(int_col1#x as int) AS int_col1#x] + : +- Project [int_col1#x] + : +- SubqueryAlias t1 + : +- LocalRelation [int_col1#x] + +- SubqueryAlias t2 + +- View (`t2`, [int_col1#x]) + +- Project [cast(int_col1#x as int) AS int_col1#x] + +- Project [int_col1#x] + +- SubqueryAlias t2 + +- LocalRelation [int_col1#x] + + +-- !query +set spark.sql.crossJoin.enabled = false +-- !query analysis +SetCommand (spark.sql.crossJoin.enabled,Some(false)) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-pivot.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-pivot.sql.out new file mode 100644 index 0000000000000..f7ef02fe50781 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-pivot.sql.out @@ -0,0 +1,734 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view courseSales as select * from values + ("dotNET", 2012, 10000), + ("Java", 2012, 20000), + ("dotNET", 2012, 5000), + ("dotNET", 2013, 48000), + ("Java", 2013, 30000) + as courseSales(course, year, earnings) +-- !query analysis +CreateViewCommand `courseSales`, select * from values + ("dotNET", 2012, 10000), + ("Java", 2012, 20000), + ("dotNET", 2012, 5000), + ("dotNET", 2013, 48000), + ("Java", 2013, 30000) + as courseSales(course, year, earnings), false, false, LocalTempView, true + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +create temporary view years as select * from values + (2012, 1), + (2013, 2) + as years(y, s) +-- !query analysis +CreateViewCommand `years`, select * from values + (2012, 1), + (2013, 2) + as years(y, s), false, false, LocalTempView, true + +- Project [y#x, s#x] + +- SubqueryAlias years + +- LocalRelation [y#x, s#x] + + +-- !query +create temporary view yearsWithComplexTypes as select * from values + (2012, array(1, 1), map('1', 1), struct(1, 'a')), + (2013, array(2, 2), map('2', 2), struct(2, 'b')) + as yearsWithComplexTypes(y, a, m, s) +-- !query analysis +CreateViewCommand `yearsWithComplexTypes`, select * from values + (2012, array(1, 1), map('1', 1), struct(1, 'a')), + (2013, array(2, 2), map('2', 2), struct(2, 'b')) + as yearsWithComplexTypes(y, a, m, s), false, false, LocalTempView, true + +- Project [y#x, a#x, m#x, s#x] + +- SubqueryAlias yearsWithComplexTypes + +- LocalRelation [y#x, a#x, m#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT udf(year), course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +Project [udf(year)#x, dotNET#xL, Java#xL] ++- Project [udf(year)#x, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[0] AS dotNET#xL, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[1] AS Java#xL] + +- Aggregate [udf(year)#x], [udf(year)#x, pivotfirst(course#x, CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL, dotNET, Java, 0, 0) AS __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x] + +- Aggregate [udf(year)#x, course#x], [udf(year)#x, course#x, cast(udf(cast(sum(earnings#x) as string)) as bigint) AS CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [cast(udf(cast(year#x as string)) as int) AS udf(year)#x, course#x, earnings#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +) +-- !query analysis +Project [course#x, 2012#xL, 2013#xL] ++- Project [course#x, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[0] AS 2012#xL, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[1] AS 2013#xL] + +- Aggregate [course#x], [course#x, pivotfirst(year#x, CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL, 2012, 2013, 0, 0) AS __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x] + +- Aggregate [course#x, year#x], [course#x, year#x, cast(udf(cast(sum(earnings#x) as string)) as bigint) AS CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL] + +- 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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), udf(avg(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +Project [year#x, dotNET_udf(sum(earnings))#xL, dotNET_udf(avg(earnings))#x, Java_udf(sum(earnings))#xL, Java_udf(avg(earnings))#x] ++- Project [year#x, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[0] AS dotNET_udf(sum(earnings))#xL, __pivot_CAST(udf(cast(avg(earnings) as string)) AS DOUBLE) AS `CAST(udf(cast(avg(earnings) as string)) AS DOUBLE)`#x[0] AS dotNET_udf(avg(earnings))#x, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[1] AS Java_udf(sum(earnings))#xL, __pivot_CAST(udf(cast(avg(earnings) as string)) AS DOUBLE) AS `CAST(udf(cast(avg(earnings) as string)) AS DOUBLE)`#x[1] AS Java_udf(avg(earnings))#x] + +- Aggregate [year#x], [year#x, pivotfirst(course#x, CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL, dotNET, Java, 0, 0) AS __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x, pivotfirst(course#x, CAST(udf(cast(avg(earnings) as string)) AS DOUBLE)#x, dotNET, Java, 0, 0) AS __pivot_CAST(udf(cast(avg(earnings) as string)) AS DOUBLE) AS `CAST(udf(cast(avg(earnings) as string)) AS DOUBLE)`#x] + +- Aggregate [year#x, course#x], [year#x, course#x, cast(udf(cast(sum(earnings#x) as string)) as bigint) AS CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL, cast(udf(cast(avg(earnings#x) as string)) as double) AS CAST(udf(cast(avg(earnings) as string)) AS DOUBLE)#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [year#x, course#x, earnings#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT * FROM ( + SELECT udf(course) as course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +Project [dotNET#xL, Java#xL] ++- Project [__pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[0] AS dotNET#xL, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[1] AS Java#xL] + +- Aggregate [pivotfirst(course#x, CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL, dotNET, Java, 0, 0) AS __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x] + +- Aggregate [course#x], [course#x, cast(udf(cast(sum(earnings#x) as string)) as bigint) AS CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [cast(udf(cast(course#x as string)) as string) AS course#x, earnings#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(udf(earnings))), udf(min(year)) + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +Project [dotNET_udf(sum(udf(earnings)))#xL, dotNET_udf(min(year))#x, Java_udf(sum(udf(earnings)))#xL, Java_udf(min(year))#x] ++- Project [__pivot_CAST(udf(cast(sum(cast(udf(cast(earnings as string)) as int)) as string)) AS BIGINT) AS `CAST(udf(cast(sum(cast(udf(cast(earnings as string)) as int)) as string)) AS BIGINT)`#x[0] AS dotNET_udf(sum(udf(earnings)))#xL, __pivot_CAST(udf(cast(min(year) as string)) AS INT) AS `CAST(udf(cast(min(year) as string)) AS INT)`#x[0] AS dotNET_udf(min(year))#x, __pivot_CAST(udf(cast(sum(cast(udf(cast(earnings as string)) as int)) as string)) AS BIGINT) AS `CAST(udf(cast(sum(cast(udf(cast(earnings as string)) as int)) as string)) AS BIGINT)`#x[1] AS Java_udf(sum(udf(earnings)))#xL, __pivot_CAST(udf(cast(min(year) as string)) AS INT) AS `CAST(udf(cast(min(year) as string)) AS INT)`#x[1] AS Java_udf(min(year))#x] + +- Aggregate [pivotfirst(course#x, CAST(udf(cast(sum(cast(udf(cast(earnings as string)) as int)) as string)) AS BIGINT)#xL, dotNET, Java, 0, 0) AS __pivot_CAST(udf(cast(sum(cast(udf(cast(earnings as string)) as int)) as string)) AS BIGINT) AS `CAST(udf(cast(sum(cast(udf(cast(earnings as string)) as int)) as string)) AS BIGINT)`#x, pivotfirst(course#x, CAST(udf(cast(min(year) as string)) AS INT)#x, dotNET, Java, 0, 0) AS __pivot_CAST(udf(cast(min(year) as string)) AS INT) AS `CAST(udf(cast(min(year) as string)) AS INT)`#x] + +- Aggregate [course#x], [course#x, cast(udf(cast(sum(cast(udf(cast(earnings#x as string)) as int)) as string)) as bigint) AS CAST(udf(cast(sum(cast(udf(cast(earnings as string)) as int)) as string)) AS BIGINT)#xL, cast(udf(cast(min(year#x) as string)) as int) AS CAST(udf(cast(min(year) as string)) AS INT)#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [year#x, course#x, earnings#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, udf(s) as s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR s IN (1, 2) +) +-- !query analysis +Project [course#x, year#x, 1#xL, 2#xL] ++- Project [course#x, year#x, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[0] AS 1#xL, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[1] AS 2#xL] + +- Aggregate [course#x, year#x], [course#x, year#x, pivotfirst(s#x, CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL, 1, 2, 0, 0) AS __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x] + +- Aggregate [course#x, year#x, s#x], [course#x, year#x, s#x, cast(udf(cast(sum(earnings#x) as string)) as bigint) AS CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, year#x, earnings#x, cast(udf(cast(s#x as string)) as int) AS s#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias years + +- View (`years`, [y#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(s#x as int) AS s#x] + +- Project [y#x, s#x] + +- SubqueryAlias years + +- LocalRelation [y#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)), udf(min(s)) + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +Project [year#x, dotNET_udf(sum(earnings))#xL, dotNET_udf(min(s))#x, Java_udf(sum(earnings))#xL, Java_udf(min(s))#x] ++- Project [year#x, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[0] AS dotNET_udf(sum(earnings))#xL, __pivot_CAST(udf(cast(min(s) as string)) AS INT) AS `CAST(udf(cast(min(s) as string)) AS INT)`#x[0] AS dotNET_udf(min(s))#x, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[1] AS Java_udf(sum(earnings))#xL, __pivot_CAST(udf(cast(min(s) as string)) AS INT) AS `CAST(udf(cast(min(s) as string)) AS INT)`#x[1] AS Java_udf(min(s))#x] + +- Aggregate [year#x], [year#x, pivotfirst(course#x, CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL, dotNET, Java, 0, 0) AS __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x, pivotfirst(course#x, CAST(udf(cast(min(s) as string)) AS INT)#x, dotNET, Java, 0, 0) AS __pivot_CAST(udf(cast(min(s) as string)) AS INT) AS `CAST(udf(cast(min(s) as string)) AS INT)`#x] + +- Aggregate [year#x, course#x], [year#x, course#x, cast(udf(cast(sum(earnings#x) as string)) as bigint) AS CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL, cast(udf(cast(min(s#x) as string)) as int) AS CAST(udf(cast(min(s) as string)) AS INT)#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, year#x, earnings#x, s#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias years + +- View (`years`, [y#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(s#x as int) AS s#x] + +- Project [y#x, s#x] + +- SubqueryAlias years + +- LocalRelation [y#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings * s)) + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +Project [year#x, dotNET#xL, Java#xL] ++- Project [year#x, __pivot_CAST(udf(cast(sum((earnings * s)) as string)) AS BIGINT) AS `CAST(udf(cast(sum((earnings * s)) as string)) AS BIGINT)`#x[0] AS dotNET#xL, __pivot_CAST(udf(cast(sum((earnings * s)) as string)) AS BIGINT) AS `CAST(udf(cast(sum((earnings * s)) as string)) AS BIGINT)`#x[1] AS Java#xL] + +- Aggregate [year#x], [year#x, pivotfirst(course#x, CAST(udf(cast(sum((earnings * s)) as string)) AS BIGINT)#xL, dotNET, Java, 0, 0) AS __pivot_CAST(udf(cast(sum((earnings * s)) as string)) AS BIGINT) AS `CAST(udf(cast(sum((earnings * s)) as string)) AS BIGINT)`#x] + +- Aggregate [year#x, course#x], [year#x, course#x, cast(udf(cast(sum((earnings#x * s#x)) as string)) as bigint) AS CAST(udf(cast(sum((earnings * s)) as string)) AS BIGINT)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, year#x, earnings#x, s#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias years + +- View (`years`, [y#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(s#x as int) AS s#x] + +- Project [y#x, s#x] + +- SubqueryAlias years + +- LocalRelation [y#x, s#x] + + +-- !query +SELECT 2012_s, 2013_s, 2012_a, 2013_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + udf(sum(e)) s, udf(avg(e)) a + FOR y IN (2012, 2013) +) +-- !query analysis +Project [2012_s#xL, 2013_s#xL, 2012_a#x, 2013_a#x, c#x] ++- Project [c#x, __pivot_CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s AS `CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s`#x[0] AS 2012_s#xL, __pivot_CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a AS `CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a`#x[0] AS 2012_a#x, __pivot_CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s AS `CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s`#x[1] AS 2013_s#xL, __pivot_CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a AS `CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a`#x[1] AS 2013_a#x] + +- Aggregate [c#x], [c#x, pivotfirst(y#x, CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s#xL, 2012, 2013, 0, 0) AS __pivot_CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s AS `CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s`#x, pivotfirst(y#x, CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a#x, 2012, 2013, 0, 0) AS __pivot_CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a AS `CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a`#x] + +- Aggregate [c#x, y#x], [c#x, y#x, cast(udf(cast(sum(e#x) as string)) as bigint) AS CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s#xL, cast(udf(cast(avg(e#x) as string)) as double) AS CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [year#x AS y#x, course#x AS c#x, earnings#x 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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT firstYear_s, secondYear_s, firstYear_a, secondYear_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + udf(sum(e)) s, udf(avg(e)) a + FOR y IN (2012 as firstYear, 2013 secondYear) +) +-- !query analysis +Project [firstYear_s#xL, secondYear_s#xL, firstYear_a#x, secondYear_a#x, c#x] ++- Project [c#x, __pivot_CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s AS `CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s`#x[0] AS firstYear_s#xL, __pivot_CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a AS `CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a`#x[0] AS firstYear_a#x, __pivot_CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s AS `CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s`#x[1] AS secondYear_s#xL, __pivot_CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a AS `CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a`#x[1] AS secondYear_a#x] + +- Aggregate [c#x], [c#x, pivotfirst(y#x, CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s#xL, 2012, 2013, 0, 0) AS __pivot_CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s AS `CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s`#x, pivotfirst(y#x, CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a#x, 2012, 2013, 0, 0) AS __pivot_CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a AS `CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a`#x] + +- Aggregate [c#x, y#x], [c#x, y#x, cast(udf(cast(sum(e#x) as string)) as bigint) AS CAST(udf(cast(sum(e) as string)) AS BIGINT) AS s#xL, cast(udf(cast(avg(e#x) as string)) as double) AS CAST(udf(cast(avg(e) as string)) AS DOUBLE) AS a#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [year#x AS y#x, course#x AS c#x, earnings#x 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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT * FROM courseSales +PIVOT ( + udf(abs(earnings)) + FOR year IN (2012, 2013) +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1006", + "messageParameters" : { + "sql" : "coursesales.earnings" + } +} + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), year + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1006", + "messageParameters" : { + "sql" : "__auto_generated_subquery_name.year" + } +} + + +-- !query +SELECT * FROM ( + SELECT course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`year`", + "proposal" : "`__auto_generated_subquery_name`.`course`, `__auto_generated_subquery_name`.`earnings`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 118, + "fragment" : "PIVOT (\n udf(sum(earnings))\n FOR year IN (2012, 2013)\n)" + } ] +} + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(ceil(udf(sum(earnings)))), avg(earnings) + 1 as a1 + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +Project [year#x, dotNET_udf(CEIL(udf(sum(earnings))))#xL, dotNET_a1#x, Java_udf(CEIL(udf(sum(earnings))))#xL, Java_a1#x] ++- Project [year#x, __pivot_CAST(udf(cast(CEIL(cast(udf(cast(sum(earnings) as string)) as bigint)) as string)) AS BIGINT) AS `CAST(udf(cast(CEIL(cast(udf(cast(sum(earnings) as string)) as bigint)) as string)) AS BIGINT)`#x[0] AS dotNET_udf(CEIL(udf(sum(earnings))))#xL, __pivot_(avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1 AS `(avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1`#x[0] AS dotNET_a1#x, __pivot_CAST(udf(cast(CEIL(cast(udf(cast(sum(earnings) as string)) as bigint)) as string)) AS BIGINT) AS `CAST(udf(cast(CEIL(cast(udf(cast(sum(earnings) as string)) as bigint)) as string)) AS BIGINT)`#x[1] AS Java_udf(CEIL(udf(sum(earnings))))#xL, __pivot_(avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1 AS `(avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1`#x[1] AS Java_a1#x] + +- Aggregate [year#x], [year#x, pivotfirst(course#x, CAST(udf(cast(CEIL(cast(udf(cast(sum(earnings) as string)) as bigint)) as string)) AS BIGINT)#xL, dotNET, Java, 0, 0) AS __pivot_CAST(udf(cast(CEIL(cast(udf(cast(sum(earnings) as string)) as bigint)) as string)) AS BIGINT) AS `CAST(udf(cast(CEIL(cast(udf(cast(sum(earnings) as string)) as bigint)) as string)) AS BIGINT)`#x, pivotfirst(course#x, (avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1#x, dotNET, Java, 0, 0) AS __pivot_(avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1 AS `(avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1`#x] + +- Aggregate [year#x, course#x], [year#x, course#x, cast(udf(cast(CEIL(cast(udf(cast(sum(earnings#x) as string)) as bigint)) as string)) as bigint) AS CAST(udf(cast(CEIL(cast(udf(cast(sum(earnings) as string)) as bigint)) as string)) AS BIGINT)#xL, (avg(earnings#x) + cast(1 as double)) AS (avg(__auto_generated_subquery_name.earnings) + CAST(1 AS DOUBLE)) AS a1#x] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [year#x, course#x, earnings#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(udf(avg(earnings))) + FOR course IN ('dotNET', 'Java') +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NESTED_AGGREGATE_FUNCTION", + "sqlState" : "42607", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 86, + "stopIndex" : 98, + "fragment" : "avg(earnings)" + } ] +} + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) +) +-- !query analysis +Project [s#x, {dotNET, 2012}#xL, {Java, 2013}#xL] ++- Project [s#x, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[0] AS {dotNET, 2012}#xL, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[1] AS {Java, 2013}#xL] + +- Aggregate [s#x], [s#x, pivotfirst(__pivot_col#x, CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL, [dotNET,2012], [Java,2013], 0, 0) AS __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x] + +- Aggregate [s#x, named_struct(course, course#x, year, year#x)], [s#x, named_struct(course, course#x, year, year#x) AS __pivot_col#x, cast(udf(cast(sum(earnings#x) as string)) as bigint) AS CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, year#x, earnings#x, s#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias years + +- View (`years`, [y#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(s#x as int) AS s#x] + +- Project [y#x, s#x] + +- SubqueryAlias years + +- LocalRelation [y#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, s) IN (('dotNET', 2) as c1, ('Java', 1) as c2) +) +-- !query analysis +Project [year#x, c1#xL, c2#xL] ++- Project [year#x, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[0] AS c1#xL, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[1] AS c2#xL] + +- Aggregate [year#x], [year#x, pivotfirst(__pivot_col#x, CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL, [dotNET,2], [Java,1], 0, 0) AS __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x] + +- Aggregate [year#x, named_struct(course, course#x, s, s#x)], [year#x, named_struct(course, course#x, s, s#x) AS __pivot_col#x, cast(udf(cast(sum(earnings#x) as string)) as bigint) AS CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, year#x, earnings#x, s#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias years + +- View (`years`, [y#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(s#x as int) AS s#x] + +- Project [y#x, s#x] + +- SubqueryAlias years + +- LocalRelation [y#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, year) IN ('dotNET', 'Java') +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIVOT_VALUE_DATA_TYPE_MISMATCH", + "sqlState" : "42K09", + "messageParameters" : { + "pivotType" : "struct", + "value" : "dotNET", + "valueType" : "string" + } +} + + +-- !query +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (s, 2013) +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`s`", + "proposal" : "`coursesales`.`year`, `coursesales`.`course`, `coursesales`.`earnings`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 71, + "fragment" : "s" + } ] +} + + +-- !query +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (course, 2013) +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NON_LITERAL_PIVOT_VALUES", + "sqlState" : "42K08", + "messageParameters" : { + "expression" : "\"course\"" + } +} + + +-- !query +SELECT * FROM ( + SELECT earnings, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR a IN (array(1, 1), array(2, 2)) +) +-- !query analysis +Project [year#x, [1, 1]#xL, [2, 2]#xL] ++- Project [year#x, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[0] AS [1, 1]#xL, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[1] AS [2, 2]#xL] + +- Aggregate [year#x], [year#x, pivotfirst(a#x, CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL, [1,1], [2,2], 0, 0) AS __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x] + +- Aggregate [year#x, a#x], [year#x, a#x, cast(udf(cast(sum(earnings#x) as string)) as bigint) AS CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [earnings#x, year#x, a#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias yearswithcomplextypes + +- View (`yearsWithComplexTypes`, [y#x,a#x,m#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(a#x as array) AS a#x, cast(m#x as map) AS m#x, cast(s#x as struct) AS s#x] + +- Project [y#x, a#x, m#x, s#x] + +- SubqueryAlias yearsWithComplexTypes + +- LocalRelation [y#x, a#x, m#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT course, earnings, udf(year) as year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) +) +-- !query analysis +Project [year#x, {dotNET, [1, 1]}#xL, {Java, [2, 2]}#xL] ++- Project [year#x, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[0] AS {dotNET, [1, 1]}#xL, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[1] AS {Java, [2, 2]}#xL] + +- Aggregate [year#x], [year#x, pivotfirst(__pivot_col#x, CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL, [dotNET,[1,1]], [Java,[2,2]], 0, 0) AS __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x] + +- Aggregate [year#x, named_struct(course, course#x, a, a#x)], [year#x, named_struct(course, course#x, a, a#x) AS __pivot_col#x, cast(udf(cast(sum(earnings#x) as string)) as bigint) AS CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, earnings#x, cast(udf(cast(year#x as string)) as int) AS year#x, a#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias yearswithcomplextypes + +- View (`yearsWithComplexTypes`, [y#x,a#x,m#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(a#x as array) AS a#x, cast(m#x as map) AS m#x, cast(s#x as struct) AS s#x] + +- Project [y#x, a#x, m#x, s#x] + +- SubqueryAlias yearsWithComplexTypes + +- LocalRelation [y#x, a#x, m#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR s IN ((1, 'a'), (2, 'b')) +) +-- !query analysis +Project [year#x, {1, a}#xL, {2, b}#xL] ++- Project [year#x, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[0] AS {1, a}#xL, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[1] AS {2, b}#xL] + +- Aggregate [year#x], [year#x, pivotfirst(s#x, CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL, [1,a], [2,b], 0, 0) AS __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x] + +- Aggregate [year#x, s#x], [year#x, s#x, cast(udf(cast(sum(earnings#x) as string)) as bigint) AS CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [earnings#x, year#x, s#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias yearswithcomplextypes + +- View (`yearsWithComplexTypes`, [y#x,a#x,m#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(a#x as array) AS a#x, cast(m#x as map) AS m#x, cast(s#x as struct) AS s#x] + +- Project [y#x, a#x, m#x, s#x] + +- SubqueryAlias yearsWithComplexTypes + +- LocalRelation [y#x, a#x, m#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT course, earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) +) +-- !query analysis +Project [year#x, {dotNET, {1, a}}#xL, {Java, {2, b}}#xL] ++- Project [year#x, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[0] AS {dotNET, {1, a}}#xL, __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x[1] AS {Java, {2, b}}#xL] + +- Aggregate [year#x], [year#x, pivotfirst(__pivot_col#x, CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL, [dotNET,[1,a]], [Java,[2,b]], 0, 0) AS __pivot_CAST(udf(cast(sum(earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(earnings) as string)) AS BIGINT)`#x] + +- Aggregate [year#x, named_struct(course, course#x, s, s#x)], [year#x, named_struct(course, course#x, s, s#x) AS __pivot_col#x, cast(udf(cast(sum(earnings#x) as string)) as bigint) AS CAST(udf(cast(sum(earnings) as string)) AS BIGINT)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, earnings#x, year#x, s#x] + +- Join Inner, (year#x = y#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] + : +- Project [course#x, year#x, earnings#x] + : +- SubqueryAlias courseSales + : +- LocalRelation [course#x, year#x, earnings#x] + +- SubqueryAlias yearswithcomplextypes + +- View (`yearsWithComplexTypes`, [y#x,a#x,m#x,s#x]) + +- Project [cast(y#x as int) AS y#x, cast(a#x as array) AS a#x, cast(m#x as map) AS m#x, cast(s#x as struct) AS s#x] + +- Project [y#x, a#x, m#x, s#x] + +- SubqueryAlias yearsWithComplexTypes + +- LocalRelation [y#x, a#x, m#x, s#x] + + +-- !query +SELECT * FROM ( + SELECT earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR m IN (map('1', 1), map('2', 2)) +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPARABLE_PIVOT_COLUMN", + "sqlState" : "42818", + "messageParameters" : { + "columnName" : "`__auto_generated_subquery_name`.`m`" + } +} + + +-- !query +SELECT * FROM ( + SELECT course, earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, m) IN (('dotNET', map('1', 1)), ('Java', map('2', 2))) +) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPARABLE_PIVOT_COLUMN", + "sqlState" : "42818", + "messageParameters" : { + "columnName" : "`named_struct('course', __auto_generated_subquery_name`.`course, 'm', __auto_generated_subquery_name`.`m)`" + } +} + + +-- !query +SELECT * FROM ( + SELECT course, earnings, udf("a") as a, udf("z") as z, udf("b") as b, udf("y") as y, + udf("c") as c, udf("x") as x, udf("d") as d, udf("w") as w + FROM courseSales +) +PIVOT ( + udf(sum(Earnings)) + FOR Course IN ('dotNET', 'Java') +) +-- !query analysis +Project [a#x, z#x, b#x, y#x, c#x, x#x, d#x, w#x, dotNET#xL, Java#xL] ++- Project [a#x, z#x, b#x, y#x, c#x, x#x, d#x, w#x, __pivot_CAST(udf(cast(sum(Earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(Earnings) as string)) AS BIGINT)`#x[0] AS dotNET#xL, __pivot_CAST(udf(cast(sum(Earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(Earnings) as string)) AS BIGINT)`#x[1] AS Java#xL] + +- Aggregate [a#x, z#x, b#x, y#x, c#x, x#x, d#x, w#x], [a#x, z#x, b#x, y#x, c#x, x#x, d#x, w#x, pivotfirst(Course#x, CAST(udf(cast(sum(Earnings) as string)) AS BIGINT)#xL, dotNET, Java, 0, 0) AS __pivot_CAST(udf(cast(sum(Earnings) as string)) AS BIGINT) AS `CAST(udf(cast(sum(Earnings) as string)) AS BIGINT)`#x] + +- Aggregate [a#x, z#x, b#x, y#x, c#x, x#x, d#x, w#x, Course#x], [a#x, z#x, b#x, y#x, c#x, x#x, d#x, w#x, Course#x, cast(udf(cast(sum(Earnings#x) as string)) as bigint) AS CAST(udf(cast(sum(Earnings) as string)) AS BIGINT)#xL] + +- SubqueryAlias __auto_generated_subquery_name + +- Project [course#x, earnings#x, cast(udf(cast(a as string)) as string) AS a#x, cast(udf(cast(z as string)) as string) AS z#x, cast(udf(cast(b as string)) as string) AS b#x, cast(udf(cast(y as string)) as string) AS y#x, cast(udf(cast(c as string)) as string) AS c#x, cast(udf(cast(x as string)) as string) AS x#x, cast(udf(cast(d as string)) as string) AS d#x, cast(udf(cast(w as string)) as string) AS w#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] + +- Project [course#x, year#x, earnings#x] + +- SubqueryAlias courseSales + +- LocalRelation [course#x, year#x, earnings#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-special-values.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-special-values.sql.out new file mode 100644 index 0000000000000..56a95930572a5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-special-values.sql.out @@ -0,0 +1,53 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT udf(x) FROM (VALUES (1), (2), (NULL)) v(x) +-- !query analysis +Project [cast(udf(cast(x#x as string)) as int) AS udf(x)#x] ++- SubqueryAlias v + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT udf(x) FROM (VALUES ('A'), ('B'), (NULL)) v(x) +-- !query analysis +Project [cast(udf(cast(x#x as string)) as string) AS udf(x)#x] ++- SubqueryAlias v + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT udf(x) FROM (VALUES ('NaN'), ('1'), ('2')) v(x) +-- !query analysis +Project [cast(udf(cast(x#x as string)) as string) AS udf(x)#x] ++- SubqueryAlias v + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT udf(x) FROM (VALUES ('Infinity'), ('1'), ('2')) v(x) +-- !query analysis +Project [cast(udf(cast(x#x as string)) as string) AS udf(x)#x] ++- SubqueryAlias v + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT udf(x) FROM (VALUES ('-Infinity'), ('1'), ('2')) v(x) +-- !query analysis +Project [cast(udf(cast(x#x as string)) as string) AS udf(x)#x] ++- SubqueryAlias v + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] + + +-- !query +SELECT udf(x) FROM (VALUES 0.00000001, 0.00000002, 0.00000003) v(x) +-- !query analysis +Project [cast(udf(cast(x#x as string)) as decimal(8,8)) AS udf(x)#x] ++- SubqueryAlias v + +- Project [col1#x AS x#x] + +- LocalRelation [col1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-udaf.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-udaf.sql.out new file mode 100644 index 0000000000000..27b9786160c45 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-udaf.sql.out @@ -0,0 +1,91 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(1), (2), (3), (4) +as t1(int_col1) +-- !query analysis +CreateViewCommand `t1`, SELECT * FROM VALUES +(1), (2), (3), (4) +as t1(int_col1), false, true, LocalTempView, true + +- Project [int_col1#x] + +- SubqueryAlias t1 + +- LocalRelation [int_col1#x] + + +-- !query +CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query analysis +CreateFunctionCommand spark_catalog.default.myDoubleAvg, test.org.apache.spark.sql.MyDoubleAvg, false, false, false + + +-- !query +SELECT default.myDoubleAvg(udf(int_col1)) as my_avg, udf(default.myDoubleAvg(udf(int_col1))) as my_avg2, udf(default.myDoubleAvg(int_col1)) as my_avg3 from t1 +-- !query analysis +Aggregate [spark_catalog.default.mydoubleavg(cast(cast(udf(cast(int_col1#x as string)) as int) as double), test.org.apache.spark.sql.MyDoubleAvg@xxxxxxxx, 0, 0, Some(spark_catalog.default.mydoubleavg)) AS my_avg#x, cast(udf(cast(spark_catalog.default.mydoubleavg(cast(cast(udf(cast(int_col1#x as string)) as int) as double), test.org.apache.spark.sql.MyDoubleAvg@xxxxxxxx, 0, 0, Some(spark_catalog.default.mydoubleavg)) as string)) as double) AS my_avg2#x, cast(udf(cast(spark_catalog.default.mydoubleavg(cast(int_col1#x as double), test.org.apache.spark.sql.MyDoubleAvg@xxxxxxxx, 0, 0, Some(spark_catalog.default.mydoubleavg)) as string)) as double) AS my_avg3#x] ++- SubqueryAlias t1 + +- View (`t1`, [int_col1#x]) + +- Project [cast(int_col1#x as int) AS int_col1#x] + +- Project [int_col1#x] + +- SubqueryAlias t1 + +- LocalRelation [int_col1#x] + + +-- !query +SELECT default.myDoubleAvg(udf(int_col1), udf(3)) as my_avg from t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`spark_catalog`.`default`.`mydoubleavg`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "default.myDoubleAvg(udf(int_col1), udf(3))" + } ] +} + + +-- !query +CREATE FUNCTION udaf1 AS 'test.non.existent.udaf' +-- !query analysis +CreateFunctionCommand spark_catalog.default.udaf1, test.non.existent.udaf, false, false, false + + +-- !query +SELECT default.udaf1(udf(int_col1)) as udaf1, udf(default.udaf1(udf(int_col1))) as udaf2, udf(default.udaf1(int_col1)) as udaf3 from t1 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CANNOT_LOAD_FUNCTION_CLASS", + "messageParameters" : { + "className" : "test.non.existent.udaf", + "functionName" : "`spark_catalog`.`default`.`udaf1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 95, + "stopIndex" : 117, + "fragment" : "default.udaf1(int_col1)" + } ] +} + + +-- !query +DROP FUNCTION myDoubleAvg +-- !query analysis +DropFunctionCommand spark_catalog.default.mydoubleavg, false, false + + +-- !query +DROP FUNCTION udaf1 +-- !query analysis +DropFunctionCommand spark_catalog.default.udaf1, false, false 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 new file mode 100644 index 0000000000000..12f07595399cd --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-union.sql.out @@ -0,0 +1,211 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (1, 'a'), (2, 'b') tbl(c1, c2) +-- !query analysis +CreateViewCommand `t1`, VALUES (1, 'a'), (2, 'b') tbl(c1, c2), false, true, LocalTempView, true + +- SubqueryAlias tbl + +- LocalRelation [c1#x, c2#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (1.0, 1), (2.0, 4) tbl(c1, c2) +-- !query analysis +CreateViewCommand `t2`, VALUES (1.0, 1), (2.0, 4) tbl(c1, c2), false, true, LocalTempView, true + +- SubqueryAlias tbl + +- LocalRelation [c1#x, c2#x] + + +-- !query +SELECT udf(c1) as c1, udf(c2) as c2 +FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 + UNION ALL + SELECT udf(c1) as c1, udf(c2) as c2 FROM t1) +-- !query analysis +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 __auto_generated_subquery_name + +- Union false, false + :- 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(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] + + +-- !query +SELECT udf(c1) as c1, udf(c2) as c2 +FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 + 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] ++- 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(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] + : +- 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(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] + + +-- !query +SELECT udf(udf(a)) as a +FROM (SELECT udf(0) a, udf(0) b + UNION ALL + SELECT udf(SUM(1)) a, udf(CAST(0 AS BIGINT)) b + UNION ALL SELECT udf(0) a, udf(0) b) T +-- !query analysis +Project [cast(udf(cast(cast(udf(cast(a#xL as string)) as bigint) as string)) as bigint) AS a#xL] ++- SubqueryAlias T + +- Union false, false + :- Union false, false + : :- Project [cast(a#x as bigint) AS a#xL, cast(b#x as bigint) AS b#xL] + : : +- Project [cast(udf(cast(0 as string)) as int) AS a#x, cast(udf(cast(0 as string)) as int) AS b#x] + : : +- OneRowRelation + : +- Aggregate [cast(udf(cast(sum(1) as string)) as bigint) AS a#xL, cast(udf(cast(cast(0 as bigint) as string)) as bigint) AS b#xL] + : +- OneRowRelation + +- Project [cast(a#x as bigint) AS a#xL, cast(b#x as bigint) AS b#xL] + +- Project [cast(udf(cast(0 as string)) as int) AS a#x, cast(udf(cast(0 as string)) as int) AS b#x] + +- OneRowRelation + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW p1 AS VALUES 1 T(col) +-- !query analysis +CreateViewCommand `p1`, VALUES 1 T(col), false, true, LocalTempView, true + +- SubqueryAlias T + +- LocalRelation [col#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW p2 AS VALUES 1 T(col) +-- !query analysis +CreateViewCommand `p2`, VALUES 1 T(col), false, true, LocalTempView, true + +- SubqueryAlias T + +- LocalRelation [col#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW p3 AS VALUES 1 T(col) +-- !query analysis +CreateViewCommand `p3`, VALUES 1 T(col), false, true, LocalTempView, true + +- SubqueryAlias T + +- LocalRelation [col#x] + + +-- !query +SELECT udf(1) AS x, + udf(col) as col +FROM (SELECT udf(col) AS col + FROM (SELECT udf(p1.col) AS col + FROM p1 CROSS JOIN p2 + UNION ALL + SELECT udf(col) + FROM p3) T1) T2 +-- !query analysis +Project [cast(udf(cast(1 as string)) as int) AS x#x, cast(udf(cast(col#x as string)) as int) AS col#x] ++- SubqueryAlias T2 + +- Project [cast(udf(cast(col#x as string)) as int) AS col#x] + +- SubqueryAlias T1 + +- Union false, false + :- Project [cast(udf(cast(col#x as string)) as int) AS col#x] + : +- Join Cross + : :- SubqueryAlias p1 + : : +- View (`p1`, [col#x]) + : : +- Project [cast(col#x as int) AS col#x] + : : +- SubqueryAlias T + : : +- LocalRelation [col#x] + : +- SubqueryAlias p2 + : +- View (`p2`, [col#x]) + : +- Project [cast(col#x as int) AS col#x] + : +- SubqueryAlias T + : +- LocalRelation [col#x] + +- Project [cast(udf(cast(col#x as string)) as int) AS udf(col)#x] + +- SubqueryAlias p3 + +- View (`p3`, [col#x]) + +- Project [cast(col#x as int) AS col#x] + +- SubqueryAlias T + +- LocalRelation [col#x] + + +-- !query +SELECT map(1, 2), udf('str') as str +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 [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, cast(null as int)) AS map(1, 2, 3, NULL)#x, cast(udf(cast(1 as string)) as int) AS udf(1)#x] + +- OneRowRelation + + +-- !query +SELECT array(1, 2), udf('str') as str +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 [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, cast(null as int)) AS array(1, 2, 3, NULL)#x, cast(udf(cast(1 as string)) as int) AS udf(1)#x] + +- OneRowRelation + + +-- !query +DROP VIEW IF EXISTS t1 +-- !query analysis +DropTempViewCommand t1 + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query analysis +DropTempViewCommand t2 + + +-- !query +DROP VIEW IF EXISTS p1 +-- !query analysis +DropTempViewCommand p1 + + +-- !query +DROP VIEW IF EXISTS p2 +-- !query analysis +DropTempViewCommand p2 + + +-- !query +DROP VIEW IF EXISTS p3 +-- !query analysis +DropTempViewCommand p3 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-window.sql.out new file mode 100644 index 0000000000000..c00c55e6b8c73 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-window.sql.out @@ -0,0 +1,493 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), +(null, null, null, null, null, null), +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) +AS testData(val, val_long, val_double, val_date, val_timestamp, cate) +-- !query analysis +CreateViewCommand `testData`, SELECT * FROM VALUES +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), +(null, null, null, null, null, null), +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) +AS testData(val, val_long, val_double, val_date, val_timestamp, cate), false, true, LocalTempView, true + +- 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 udf(val), cate, count(val) OVER(PARTITION BY cate ORDER BY udf(val) ROWS CURRENT ROW) FROM testData +ORDER BY cate, udf(val) +-- !query analysis +Project [udf(val)#x, cate#x, count(val) OVER (PARTITION BY cate ORDER BY udf(val) ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND CURRENT ROW)#xL] ++- Sort [cate#x ASC NULLS FIRST, cast(udf(cast(val#x as string)) as int) ASC NULLS FIRST], true + +- Project [udf(val)#x, cate#x, count(val) OVER (PARTITION BY cate ORDER BY udf(val) ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND CURRENT ROW)#xL, val#x] + +- Project [udf(val)#x, cate#x, val#x, _w1#x, count(val) OVER (PARTITION BY cate ORDER BY udf(val) ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND CURRENT ROW)#xL, count(val) OVER (PARTITION BY cate ORDER BY udf(val) ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND CURRENT ROW)#xL] + +- Window [count(val#x) windowspecdefinition(cate#x, _w1#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), currentrow$())) AS count(val) OVER (PARTITION BY cate ORDER BY udf(val) ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND CURRENT ROW)#xL], [cate#x], [_w1#x ASC NULLS FIRST] + +- Project [cast(udf(cast(val#x as string)) as int) AS udf(val)#x, cate#x, val#x, cast(udf(cast(val#x as string)) as int) 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] + +- 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 udf(val), cate, sum(val) OVER(PARTITION BY cate ORDER BY udf(val) +ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) +-- !query analysis +Project [udf(val)#x, cate#x, sum(val) OVER (PARTITION BY cate ORDER BY udf(val) ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)#xL] ++- Sort [cate#x ASC NULLS FIRST, cast(udf(cast(val#x as string)) as int) ASC NULLS FIRST], true + +- Project [udf(val)#x, cate#x, sum(val) OVER (PARTITION BY cate ORDER BY udf(val) ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)#xL, val#x] + +- Project [udf(val)#x, cate#x, val#x, _w1#x, sum(val) OVER (PARTITION BY cate ORDER BY udf(val) ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)#xL, sum(val) OVER (PARTITION BY cate ORDER BY udf(val) ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)#xL] + +- Window [sum(val#x) windowspecdefinition(cate#x, _w1#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), 1)) AS sum(val) OVER (PARTITION BY cate ORDER BY udf(val) ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)#xL], [cate#x], [_w1#x ASC NULLS FIRST] + +- Project [cast(udf(cast(val#x as string)) as int) AS udf(val)#x, cate#x, val#x, cast(udf(cast(val#x as string)) as int) 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] + +- 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 val_long, udf(cate), sum(val_long) OVER(PARTITION BY cate ORDER BY udf(val_long) +ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY udf(cate), val_long +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "expectedType" : "\"INT\"", + "exprType" : "\"BIGINT\"", + "location" : "upper", + "sqlExpr" : "\"ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 47, + "stopIndex" : 138, + "fragment" : "(PARTITION BY cate ORDER BY udf(val_long)\nROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING)" + } ] +} + + +-- !query +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY val RANGE 1 PRECEDING) FROM testData +ORDER BY cate, udf(val) +-- !query analysis +Project [udf(val)#x, cate#x, count(val) OVER (PARTITION BY udf(cate) ORDER BY val ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND CURRENT ROW)#xL] ++- Sort [cate#x ASC NULLS FIRST, cast(udf(cast(val#x as string)) as int) ASC NULLS FIRST], true + +- Project [udf(val)#x, cate#x, count(val) OVER (PARTITION BY udf(cate) ORDER BY val ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND CURRENT ROW)#xL, val#x] + +- Project [udf(val)#x, cate#x, val#x, _w1#x, count(val) OVER (PARTITION BY udf(cate) ORDER BY val ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND CURRENT ROW)#xL, count(val) OVER (PARTITION BY udf(cate) ORDER BY val ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND CURRENT ROW)#xL] + +- Window [count(val#x) windowspecdefinition(_w1#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -1, currentrow$())) AS count(val) OVER (PARTITION BY udf(cate) ORDER BY val ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND CURRENT ROW)#xL], [_w1#x], [val#x ASC NULLS FIRST] + +- Project [cast(udf(cast(val#x as string)) as int) AS udf(val)#x, cate#x, val#x, cast(udf(cast(cate#x as string)) as string) 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] + +- 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 val, udf(cate), sum(val) OVER(PARTITION BY udf(cate) ORDER BY val +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY udf(cate), val +-- !query analysis +Project [val#x, udf(cate)#x, sum(val) OVER (PARTITION BY udf(cate) ORDER BY val ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] ++- Sort [cast(udf(cast(cate#x as string)) as string) ASC NULLS FIRST, val#x ASC NULLS FIRST], true + +- Project [val#x, udf(cate)#x, sum(val) OVER (PARTITION BY udf(cate) ORDER BY val ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, cate#x] + +- Project [val#x, udf(cate)#x, _w0#x, sum(val) OVER (PARTITION BY udf(cate) ORDER BY val ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, sum(val) OVER (PARTITION BY udf(cate) ORDER BY val ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, cate#x] + +- Window [sum(val#x) windowspecdefinition(_w0#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), 1)) AS sum(val) OVER (PARTITION BY udf(cate) ORDER BY val ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL], [_w0#x], [val#x ASC NULLS FIRST] + +- Project [val#x, cast(udf(cast(cate#x as string)) as string) AS udf(cate)#x, cast(udf(cast(cate#x as string)) as string) AS _w0#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 val_long, udf(cate), sum(val_long) OVER(PARTITION BY udf(cate) ORDER BY val_long +RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY udf(cate), val_long +-- !query analysis +Project [val_long#xL, udf(cate)#x, sum(val_long) OVER (PARTITION BY udf(cate) ORDER BY val_long ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING)#xL] ++- Sort [cast(udf(cast(cate#x as string)) as string) ASC NULLS FIRST, val_long#xL ASC NULLS FIRST], true + +- Project [val_long#xL, udf(cate)#x, sum(val_long) OVER (PARTITION BY udf(cate) ORDER BY val_long ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING)#xL, cate#x] + +- Project [val_long#xL, udf(cate)#x, _w0#x, sum(val_long) OVER (PARTITION BY udf(cate) ORDER BY val_long ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING)#xL, sum(val_long) OVER (PARTITION BY udf(cate) ORDER BY val_long ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING)#xL, cate#x] + +- Window [sum(val_long#xL) windowspecdefinition(_w0#x, val_long#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), 2147483648)) AS sum(val_long) OVER (PARTITION BY udf(cate) ORDER BY val_long ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING)#xL], [_w0#x], [val_long#xL ASC NULLS FIRST] + +- Project [val_long#xL, cast(udf(cast(cate#x as string)) as string) AS udf(cate)#x, cast(udf(cast(cate#x as string)) as string) AS _w0#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 val_double, udf(cate), sum(val_double) OVER(PARTITION BY udf(cate) ORDER BY val_double +RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY udf(cate), val_double +-- !query analysis +Project [val_double#x, udf(cate)#x, sum(val_double) OVER (PARTITION BY udf(cate) ORDER BY val_double ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING)#x] ++- Sort [cast(udf(cast(cate#x as string)) as string) ASC NULLS FIRST, val_double#x ASC NULLS FIRST], true + +- Project [val_double#x, udf(cate)#x, sum(val_double) OVER (PARTITION BY udf(cate) ORDER BY val_double ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING)#x, cate#x] + +- Project [val_double#x, udf(cate)#x, _w0#x, sum(val_double) OVER (PARTITION BY udf(cate) ORDER BY val_double ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING)#x, sum(val_double) OVER (PARTITION BY udf(cate) ORDER BY val_double ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING)#x, cate#x] + +- Window [sum(val_double#x) windowspecdefinition(_w0#x, val_double#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), cast(2.5 as double))) AS sum(val_double) OVER (PARTITION BY udf(cate) ORDER BY val_double ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING)#x], [_w0#x], [val_double#x ASC NULLS FIRST] + +- Project [val_double#x, cast(udf(cast(cate#x as string)) as string) AS udf(cate)#x, cast(udf(cast(cate#x as string)) as string) AS _w0#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 val_date, udf(cate), max(val_date) OVER(PARTITION BY udf(cate) ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY udf(cate), val_date +-- !query analysis +Project [val_date#x, udf(cate)#x, max(val_date) OVER (PARTITION BY udf(cate) ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING)#x] ++- Sort [cast(udf(cast(cate#x as string)) as string) ASC NULLS FIRST, val_date#x ASC NULLS FIRST], true + +- Project [val_date#x, udf(cate)#x, max(val_date) OVER (PARTITION BY udf(cate) ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING)#x, cate#x] + +- Project [val_date#x, udf(cate)#x, _w0#x, max(val_date) OVER (PARTITION BY udf(cate) ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING)#x, max(val_date) OVER (PARTITION BY udf(cate) ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING)#x, cate#x] + +- Window [max(val_date#x) windowspecdefinition(_w0#x, val_date#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), 2)) AS max(val_date) OVER (PARTITION BY udf(cate) ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING)#x], [_w0#x], [val_date#x ASC NULLS FIRST] + +- Project [val_date#x, cast(udf(cast(cate#x as string)) as string) AS udf(cate)#x, cast(udf(cast(cate#x as string)) as string) AS _w0#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 val_timestamp, udf(cate), avg(val_timestamp) OVER(PARTITION BY udf(cate) ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY udf(cate), val_timestamp +-- !query analysis +Project [val_timestamp#x, udf(cate)#x, avg(val_timestamp) OVER (PARTITION BY udf(cate) ORDER BY val_timestamp ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '23 04' DAY TO HOUR FOLLOWING)#x] ++- Sort [cast(udf(cast(cate#x as string)) as string) ASC NULLS FIRST, val_timestamp#x ASC NULLS FIRST], true + +- Project [val_timestamp#x, udf(cate)#x, avg(val_timestamp) OVER (PARTITION BY udf(cate) ORDER BY val_timestamp ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '23 04' DAY TO HOUR FOLLOWING)#x, cate#x] + +- Project [val_timestamp#x, udf(cate)#x, _w0#x, _w1#x, avg(val_timestamp) OVER (PARTITION BY udf(cate) ORDER BY val_timestamp ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '23 04' DAY TO HOUR FOLLOWING)#x, avg(val_timestamp) OVER (PARTITION BY udf(cate) ORDER BY val_timestamp ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '23 04' DAY TO HOUR FOLLOWING)#x, cate#x] + +- Window [avg(_w0#x) windowspecdefinition(_w1#x, val_timestamp#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), INTERVAL '23 04' DAY TO HOUR)) AS avg(val_timestamp) OVER (PARTITION BY udf(cate) ORDER BY val_timestamp ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '23 04' DAY TO HOUR FOLLOWING)#x], [_w1#x], [val_timestamp#x ASC NULLS FIRST] + +- Project [val_timestamp#x, cast(udf(cast(cate#x as string)) as string) AS udf(cate)#x, cast(val_timestamp#x as double) AS _w0#x, cast(udf(cast(cate#x as string)) as string) AS _w1#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 val, udf(cate), sum(val) OVER(PARTITION BY cate ORDER BY val DESC +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query analysis +Project [val#x, udf(cate)#x, sum(val) OVER (PARTITION BY cate ORDER BY val DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] ++- Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true + +- Project [val#x, udf(cate)#x, sum(val) OVER (PARTITION BY cate ORDER BY val DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, cate#x] + +- Project [val#x, udf(cate)#x, cate#x, sum(val) OVER (PARTITION BY cate ORDER BY val DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, sum(val) OVER (PARTITION BY cate ORDER BY val DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] + +- Window [sum(val#x) windowspecdefinition(cate#x, val#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), 1)) AS sum(val) OVER (PARTITION BY cate ORDER BY val DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL], [cate#x], [val#x DESC NULLS LAST] + +- Project [val#x, cast(udf(cast(cate#x as string)) as string) AS udf(cate)#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 udf(val), cate, count(val) OVER(PARTITION BY udf(cate) +ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_INVALID_BOUND", + "sqlState" : "42K09", + "messageParameters" : { + "lower" : "\"UNBOUNDED FOLLOWING\"", + "sqlExpr" : "\"ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING\"", + "upper" : "\"1\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 111, + "fragment" : "(PARTITION BY udf(cate)\nROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"(PARTITION BY udf(cate) RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 104, + "fragment" : "(PARTITION BY udf(cate)\nRANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY udf(val), cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_MULTI_ORDER", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpec" : "cast(udf(cast(val#x as string)) as int) ASC NULLS FIRST,cate#x ASC NULLS FIRST", + "sqlExpr" : "\"(PARTITION BY udf(cate) ORDER BY udf(val) ASC NULLS FIRST, cate ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 128, + "fragment" : "(PARTITION BY udf(cate) ORDER BY udf(val), cate\nRANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY current_timestamp +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpecType" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(PARTITION BY udf(cate) ORDER BY current_timestamp() ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"", + "valueBoundaryType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 131, + "fragment" : "(PARTITION BY udf(cate) ORDER BY current_timestamp\nRANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY val +RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY udf(cate), val +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_WRONG_COMPARISON", + "sqlState" : "42K09", + "messageParameters" : { + "comparison" : "less than or equal", + "sqlExpr" : "\"RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 117, + "fragment" : "(PARTITION BY udf(cate) ORDER BY val\nRANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING)" + } ] +} + + +-- !query +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY udf(val) +RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val(val) +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Frame bound value must be a literal." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 111, + "stopIndex" : 132, + "fragment" : "current_date PRECEDING" + } ] +} + + +-- !query +SELECT udf(val), cate, +max(udf(val)) OVER w AS max, +min(udf(val)) OVER w AS min, +min(udf(val)) OVER w AS min, +count(udf(val)) OVER w AS count, +sum(udf(val)) OVER w AS sum, +avg(udf(val)) OVER w AS avg, +stddev(udf(val)) OVER w AS stddev, +first_value(udf(val)) OVER w AS first_value, +first_value(udf(val), true) OVER w AS first_value_ignore_null, +first_value(udf(val), false) OVER w AS first_value_contain_null, +any_value(udf(val)) OVER w AS any_value, +any_value(udf(val), true) OVER w AS any_value_ignore_null, +any_value(udf(val), false) OVER w AS any_value_contain_null, +last_value(udf(val)) OVER w AS last_value, +last_value(udf(val), true) OVER w AS last_value_ignore_null, +last_value(udf(val), false) OVER w AS last_value_contain_null, +rank() OVER w AS rank, +dense_rank() OVER w AS dense_rank, +cume_dist() OVER w AS cume_dist, +percent_rank() OVER w AS percent_rank, +ntile(2) OVER w AS ntile, +row_number() OVER w AS row_number, +var_pop(udf(val)) OVER w AS var_pop, +var_samp(udf(val)) OVER w AS var_samp, +approx_count_distinct(udf(val)) OVER w AS approx_count_distinct, +covar_pop(udf(val), udf(val_long)) OVER w AS covar_pop, +corr(udf(val), udf(val_long)) OVER w AS corr, +stddev_samp(udf(val)) OVER w AS stddev_samp, +stddev_pop(udf(val)) OVER w AS stddev_pop, +collect_list(udf(val)) OVER w AS collect_list, +collect_set(udf(val)) OVER w AS collect_set, +skewness(udf(val_double)) OVER w AS skewness, +kurtosis(udf(val_double)) OVER w AS kurtosis +FROM testData +WINDOW w AS (PARTITION BY udf(cate) ORDER BY udf(val)) +ORDER BY cate, udf(val) +-- !query analysis +Project [udf(val)#x, cate#x, max#x, min#x, min#x, count#xL, sum#xL, avg#x, stddev#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, rank#x, dense_rank#x, cume_dist#x, percent_rank#x, ntile#x, row_number#x, ... 11 more fields] ++- Sort [cate#x ASC NULLS FIRST, cast(udf(cast(val#x as string)) as int) ASC NULLS FIRST], true + +- Project [udf(val)#x, cate#x, max#x, min#x, min#x, count#xL, sum#xL, avg#x, stddev#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, rank#x, dense_rank#x, cume_dist#x, percent_rank#x, ntile#x, row_number#x, ... 12 more fields] + +- Project [udf(val)#x, cate#x, _w0#x, _w1#x, _w2#x, _w3#x, _w4#x, max#x, min#x, min#x, count#xL, sum#xL, avg#x, stddev#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, rank#x, ... 50 more fields] + +- Window [max(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS max#x, min(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS min#x, min(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS min#x, count(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count#xL, sum(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS sum#xL, avg(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS avg#x, stddev(_w2#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS stddev#x, first_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value#x, first_value(_w0#x, true) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value_ignore_null#x, first_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value_contain_null#x, any_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value#x, any_value(_w0#x, true) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value_ignore_null#x, any_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value_contain_null#x, last_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value#x, last_value(_w0#x, true) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value_ignore_null#x, last_value(_w0#x, false) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value_contain_null#x, rank(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#x, dense_rank(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS dense_rank#x, cume_dist() windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS cume_dist#x, percent_rank(_w0#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS percent_rank#x, ntile(2) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ntile#x, row_number() windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS row_number#x, var_pop(_w2#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS var_pop#x, var_samp(_w2#x) windowspecdefinition(_w1#x, _w0#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS var_samp#x, ... 9 more fields], [_w1#x], [_w0#x ASC NULLS FIRST] + +- Project [cast(udf(cast(val#x as string)) as int) AS udf(val)#x, cate#x, cast(udf(cast(val#x as string)) as int) AS _w0#x, cast(udf(cast(cate#x as string)) as string) AS _w1#x, cast(cast(udf(cast(val#x as string)) as int) as double) AS _w2#x, cast(cast(udf(cast(val_long#xL as string)) as bigint) as double) AS _w3#x, cast(udf(cast(val_double#x as string)) as double) AS _w4#x, val#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 udf(val), cate, avg(null) OVER(PARTITION BY cate ORDER BY val) FROM testData ORDER BY cate, val +-- !query analysis +Project [udf(val)#x, cate#x, avg(NULL) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] ++- Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true + +- Project [udf(val)#x, cate#x, avg(NULL) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, val#x] + +- Project [udf(val)#x, cate#x, val#x, avg(NULL) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, avg(NULL) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [avg(cast(null as double)) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS avg(NULL) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [cate#x], [val#x ASC NULLS FIRST] + +- Project [cast(udf(cast(val#x as string)) as int) AS udf(val)#x, cate#x, val#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 udf(val), cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY cate, udf(val) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1037", + "messageParameters" : { + "wf" : "row_number()" + } +} + + +-- !query +SELECT udf(val), cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val +-- !query analysis +Project [udf(val)#x, cate#x, sum(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, avg(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] ++- Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true + +- Project [udf(val)#x, cate#x, sum(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, avg(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, val#x] + +- Project [udf(val)#x, cate#x, val#x, sum(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, avg(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, sum(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, avg(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Window [sum(val#x) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS sum(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, avg(val#x) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Project [cast(udf(cast(val#x as string)) as int) AS udf(val)#x, cate#x, val#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 udf(val), cate, +first_value(false) OVER w AS first_value, +first_value(true, true) OVER w AS first_value_ignore_null, +first_value(false, false) OVER w AS first_value_contain_null, +any_value(false) OVER w AS any_value, +any_value(true, true) OVER w AS any_value_ignore_null, +any_value(false, false) OVER w AS any_value_contain_null, +last_value(false) OVER w AS last_value, +last_value(true, true) OVER w AS last_value_ignore_null, +last_value(false, false) OVER w AS last_value_contain_null +FROM testData +WINDOW w AS () +ORDER BY cate, val +-- !query analysis +Project [udf(val)#x, cate#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x] ++- Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true + +- Project [udf(val)#x, cate#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, val#x] + +- Project [udf(val)#x, cate#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, val#x] + +- Window [first_value(false, false) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS first_value#x, first_value(true, true) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS first_value_ignore_null#x, first_value(false, false) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS first_value_contain_null#x, any_value(false, false) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS any_value#x, any_value(true, true) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS any_value_ignore_null#x, any_value(false, false) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS any_value_contain_null#x, last_value(false, false) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS last_value#x, last_value(true, true) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS last_value_ignore_null#x, last_value(false, false) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS last_value_contain_null#x] + +- Project [cast(udf(cast(val#x as string)) as int) AS udf(val)#x, cate#x, val#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 udf(cate), sum(val) OVER (w) +FROM testData +WHERE val is not null +WINDOW w AS (PARTITION BY cate ORDER BY val) +-- !query analysis +Project [udf(cate)#x, sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] ++- Project [udf(cate)#x, val#x, cate#x, sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] + +- Window [sum(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL], [cate#x], [val#x ASC NULLS FIRST] + +- Project [cast(udf(cast(cate#x as string)) as string) AS udf(cate)#x, val#x, cate#x] + +- Filter isnotnull(val#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] 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 new file mode 100644 index 0000000000000..b6d268724e739 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/union.sql.out @@ -0,0 +1,276 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (1, 'a'), (2, 'b') tbl(c1, c2) +-- !query analysis +CreateViewCommand `t1`, VALUES (1, 'a'), (2, 'b') tbl(c1, c2), false, true, LocalTempView, true + +- SubqueryAlias tbl + +- LocalRelation [c1#x, c2#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (1.0, 1), (2.0, 4) tbl(c1, c2) +-- !query analysis +CreateViewCommand `t2`, VALUES (1.0, 1), (2.0, 4) tbl(c1, c2), false, true, LocalTempView, true + +- SubqueryAlias tbl + +- LocalRelation [c1#x, c2#x] + + +-- !query +SELECT * +FROM (SELECT * FROM t1 + UNION ALL + SELECT * FROM t1) +-- !query analysis +Project [c1#x, c2#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Union false, false + :- 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 [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] + + +-- !query +SELECT * +FROM (SELECT * FROM t1 + UNION ALL + SELECT * FROM t2 + UNION ALL + SELECT * FROM t2) +-- !query analysis +Project [c1#x, c2#x] ++- 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 [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 [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 [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] + + +-- !query +SELECT a +FROM (SELECT 0 a, 0 b + UNION ALL + SELECT SUM(1) a, CAST(0 AS BIGINT) b + UNION ALL SELECT 0 a, 0 b) T +-- !query analysis +Project [a#xL] ++- SubqueryAlias T + +- Union false, false + :- Union false, false + : :- Project [cast(a#x as bigint) AS a#xL, cast(b#x as bigint) AS b#xL] + : : +- Project [0 AS a#x, 0 AS b#x] + : : +- OneRowRelation + : +- Aggregate [sum(1) AS a#xL, cast(0 as bigint) AS b#xL] + : +- OneRowRelation + +- Project [cast(a#x as bigint) AS a#xL, cast(b#x as bigint) AS b#xL] + +- Project [0 AS a#x, 0 AS b#x] + +- OneRowRelation + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW p1 AS VALUES 1 T(col) +-- !query analysis +CreateViewCommand `p1`, VALUES 1 T(col), false, true, LocalTempView, true + +- SubqueryAlias T + +- LocalRelation [col#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW p2 AS VALUES 1 T(col) +-- !query analysis +CreateViewCommand `p2`, VALUES 1 T(col), false, true, LocalTempView, true + +- SubqueryAlias T + +- LocalRelation [col#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW p3 AS VALUES 1 T(col) +-- !query analysis +CreateViewCommand `p3`, VALUES 1 T(col), false, true, LocalTempView, true + +- SubqueryAlias T + +- LocalRelation [col#x] + + +-- !query +SELECT 1 AS x, + col +FROM (SELECT col AS col + FROM (SELECT p1.col AS col + FROM p1 CROSS JOIN p2 + UNION ALL + SELECT col + FROM p3) T1) T2 +-- !query analysis +Project [1 AS x#x, col#x] ++- SubqueryAlias T2 + +- Project [col#x AS col#x] + +- SubqueryAlias T1 + +- Union false, false + :- Project [col#x AS col#x] + : +- Join Cross + : :- SubqueryAlias p1 + : : +- View (`p1`, [col#x]) + : : +- Project [cast(col#x as int) AS col#x] + : : +- SubqueryAlias T + : : +- LocalRelation [col#x] + : +- SubqueryAlias p2 + : +- View (`p2`, [col#x]) + : +- Project [cast(col#x as int) AS col#x] + : +- SubqueryAlias T + : +- LocalRelation [col#x] + +- Project [col#x] + +- SubqueryAlias p3 + +- View (`p3`, [col#x]) + +- Project [cast(col#x as int) AS col#x] + +- SubqueryAlias T + +- LocalRelation [col#x] + + +-- !query +SELECT map(1, 2), 'str' +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 [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, cast(null as int)) AS map(1, 2, 3, NULL)#x, 1 AS 1#x] + +- OneRowRelation + + +-- !query +SELECT array(1, 2), 'str' +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 [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, cast(null as int)) AS array(1, 2, 3, NULL)#x, 1 AS 1#x] + +- OneRowRelation + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t3 AS VALUES (decimal(1)) tbl(v) +-- !query analysis +CreateViewCommand `t3`, VALUES (decimal(1)) tbl(v), false, true, LocalTempView, true + +- SubqueryAlias tbl + +- LocalRelation [v#x] + + +-- !query +SELECT t.v FROM ( + SELECT v FROM t3 + UNION ALL + SELECT v + v AS v FROM t3 +) t +-- !query analysis +Project [v#x] ++- SubqueryAlias t + +- Union false, false + :- Project [cast(v#x as decimal(11,0)) AS v#x] + : +- Project [v#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [v#x]) + : +- Project [cast(v#x as decimal(10,0)) AS v#x] + : +- SubqueryAlias tbl + : +- LocalRelation [v#x] + +- Project [(v#x + v#x) AS v#x] + +- SubqueryAlias t3 + +- View (`t3`, [v#x]) + +- Project [cast(v#x as decimal(10,0)) AS v#x] + +- SubqueryAlias tbl + +- LocalRelation [v#x] + + +-- !query +SELECT SUM(t.v) FROM ( + SELECT v FROM t3 + UNION + SELECT v + v AS v FROM t3 +) t +-- !query analysis +Aggregate [sum(v#x) AS sum(v)#x] ++- SubqueryAlias t + +- Distinct + +- Union false, false + :- Project [cast(v#x as decimal(11,0)) AS v#x] + : +- Project [v#x] + : +- SubqueryAlias t3 + : +- View (`t3`, [v#x]) + : +- Project [cast(v#x as decimal(10,0)) AS v#x] + : +- SubqueryAlias tbl + : +- LocalRelation [v#x] + +- Project [(v#x + v#x) AS v#x] + +- SubqueryAlias t3 + +- View (`t3`, [v#x]) + +- Project [cast(v#x as decimal(10,0)) AS v#x] + +- SubqueryAlias tbl + +- LocalRelation [v#x] + + +-- !query +DROP VIEW IF EXISTS t1 +-- !query analysis +DropTempViewCommand t1 + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query analysis +DropTempViewCommand t2 + + +-- !query +DROP VIEW IF EXISTS t3 +-- !query analysis +DropTempViewCommand t3 + + +-- !query +DROP VIEW IF EXISTS p1 +-- !query analysis +DropTempViewCommand p1 + + +-- !query +DROP VIEW IF EXISTS p2 +-- !query analysis +DropTempViewCommand p2 + + +-- !query +DROP VIEW IF EXISTS p3 +-- !query analysis +DropTempViewCommand p3 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/unpivot.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/unpivot.sql.out new file mode 100644 index 0000000000000..7663f56acd476 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/unpivot.sql.out @@ -0,0 +1,129 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view courseEarnings as select * from values + ("dotNET", 15000, 48000, 22500), + ("Java", 20000, 30000, NULL) + as courseEarnings(course, `2012`, `2013`, `2014`) +-- !query analysis +CreateViewCommand `courseEarnings`, select * from values + ("dotNET", 15000, 48000, 22500), + ("Java", 20000, 30000, NULL) + as courseEarnings(course, `2012`, `2013`, `2014`), false, false, LocalTempView, true + +- Project [course#x, 2012#x, 2013#x, 2014#x] + +- SubqueryAlias courseEarnings + +- LocalRelation [course#x, 2012#x, 2013#x, 2014#x] + + +-- !query +SELECT * FROM courseEarnings +UNPIVOT ( + earningsYear FOR year IN (`2012`, `2013`, `2014`) +) +-- !query analysis +Project [course#x, year#x, earningsYear#x] ++- Filter isnotnull(coalesce(earningsYear#x)) + +- Expand [[course#x, 2012, 2012#x], [course#x, 2013, 2013#x], [course#x, 2014, 2014#x]], [course#x, year#x, earningsYear#x] + +- SubqueryAlias courseearnings + +- View (`courseEarnings`, [course#x,2012#x,2013#x,2014#x]) + +- Project [cast(course#x as string) AS course#x, cast(2012#x as int) AS 2012#x, cast(2013#x as int) AS 2013#x, cast(2014#x as int) AS 2014#x] + +- Project [course#x, 2012#x, 2013#x, 2014#x] + +- SubqueryAlias courseEarnings + +- LocalRelation [course#x, 2012#x, 2013#x, 2014#x] + + +-- !query +SELECT * FROM courseEarnings +UNPIVOT INCLUDE NULLS ( + earningsYear FOR year IN (`2012`, `2013`, `2014`) +) +-- !query analysis +Project [course#x, year#x, earningsYear#x] ++- Expand [[course#x, 2012, 2012#x], [course#x, 2013, 2013#x], [course#x, 2014, 2014#x]], [course#x, year#x, earningsYear#x] + +- SubqueryAlias courseearnings + +- View (`courseEarnings`, [course#x,2012#x,2013#x,2014#x]) + +- Project [cast(course#x as string) AS course#x, cast(2012#x as int) AS 2012#x, cast(2013#x as int) AS 2013#x, cast(2014#x as int) AS 2014#x] + +- Project [course#x, 2012#x, 2013#x, 2014#x] + +- SubqueryAlias courseEarnings + +- LocalRelation [course#x, 2012#x, 2013#x, 2014#x] + + +-- !query +SELECT * FROM courseEarnings +UNPIVOT ( + earningsYear FOR year IN (`2012` as `twenty-twelve`, `2013` as `twenty-thirteen`, `2014` as `twenty-fourteen`) +) +-- !query analysis +Project [course#x, year#x, earningsYear#x] ++- Filter isnotnull(coalesce(earningsYear#x)) + +- Expand [[course#x, twenty-twelve, 2012#x], [course#x, twenty-thirteen, 2013#x], [course#x, twenty-fourteen, 2014#x]], [course#x, year#x, earningsYear#x] + +- SubqueryAlias courseearnings + +- View (`courseEarnings`, [course#x,2012#x,2013#x,2014#x]) + +- Project [cast(course#x as string) AS course#x, cast(2012#x as int) AS 2012#x, cast(2013#x as int) AS 2013#x, cast(2014#x as int) AS 2014#x] + +- Project [course#x, 2012#x, 2013#x, 2014#x] + +- SubqueryAlias courseEarnings + +- LocalRelation [course#x, 2012#x, 2013#x, 2014#x] + + +-- !query +create temporary view courseEarningsAndSales as select * from values + ("dotNET", 15000, NULL, 48000, 1, 22500, 1), + ("Java", 20000, 1, 30000, 2, NULL, NULL) + as courseEarningsAndSales(course, earnings2012, sales2012, earnings2013, sales2013, earnings2014, sales2014) +-- !query analysis +CreateViewCommand `courseEarningsAndSales`, select * from values + ("dotNET", 15000, NULL, 48000, 1, 22500, 1), + ("Java", 20000, 1, 30000, 2, NULL, NULL) + as courseEarningsAndSales(course, earnings2012, sales2012, earnings2013, sales2013, earnings2014, sales2014), false, false, LocalTempView, true + +- Project [course#x, earnings2012#x, sales2012#x, earnings2013#x, sales2013#x, earnings2014#x, sales2014#x] + +- SubqueryAlias courseEarningsAndSales + +- LocalRelation [course#x, earnings2012#x, sales2012#x, earnings2013#x, sales2013#x, earnings2014#x, sales2014#x] + + +-- !query +SELECT * FROM courseEarningsAndSales +UNPIVOT ( + (earnings, sales) FOR year IN ((earnings2012, sales2012), (earnings2013, sales2013), (earnings2014, sales2014)) +) +-- !query analysis +Project [course#x, year#x, earnings#x, sales#x] ++- Filter isnotnull(coalesce(earnings#x, sales#x)) + +- Expand [[course#x, earnings2012_sales2012, earnings2012#x, sales2012#x], [course#x, earnings2013_sales2013, earnings2013#x, sales2013#x], [course#x, earnings2014_sales2014, earnings2014#x, sales2014#x]], [course#x, year#x, earnings#x, sales#x] + +- SubqueryAlias courseearningsandsales + +- View (`courseEarningsAndSales`, [course#x,earnings2012#x,sales2012#x,earnings2013#x,sales2013#x,earnings2014#x,sales2014#x]) + +- Project [cast(course#x as string) AS course#x, cast(earnings2012#x as int) AS earnings2012#x, cast(sales2012#x as int) AS sales2012#x, cast(earnings2013#x as int) AS earnings2013#x, cast(sales2013#x as int) AS sales2013#x, cast(earnings2014#x as int) AS earnings2014#x, cast(sales2014#x as int) AS sales2014#x] + +- Project [course#x, earnings2012#x, sales2012#x, earnings2013#x, sales2013#x, earnings2014#x, sales2014#x] + +- SubqueryAlias courseEarningsAndSales + +- LocalRelation [course#x, earnings2012#x, sales2012#x, earnings2013#x, sales2013#x, earnings2014#x, sales2014#x] + + +-- !query +SELECT * FROM courseEarningsAndSales +UNPIVOT INCLUDE NULLS ( + (earnings, sales) FOR year IN ((earnings2012, sales2012), (earnings2013, sales2013), (earnings2014, sales2014)) +) +-- !query analysis +Project [course#x, year#x, earnings#x, sales#x] ++- Expand [[course#x, earnings2012_sales2012, earnings2012#x, sales2012#x], [course#x, earnings2013_sales2013, earnings2013#x, sales2013#x], [course#x, earnings2014_sales2014, earnings2014#x, sales2014#x]], [course#x, year#x, earnings#x, sales#x] + +- SubqueryAlias courseearningsandsales + +- View (`courseEarningsAndSales`, [course#x,earnings2012#x,sales2012#x,earnings2013#x,sales2013#x,earnings2014#x,sales2014#x]) + +- Project [cast(course#x as string) AS course#x, cast(earnings2012#x as int) AS earnings2012#x, cast(sales2012#x as int) AS sales2012#x, cast(earnings2013#x as int) AS earnings2013#x, cast(sales2013#x as int) AS sales2013#x, cast(earnings2014#x as int) AS earnings2014#x, cast(sales2014#x as int) AS sales2014#x] + +- Project [course#x, earnings2012#x, sales2012#x, earnings2013#x, sales2013#x, earnings2014#x, sales2014#x] + +- SubqueryAlias courseEarningsAndSales + +- LocalRelation [course#x, earnings2012#x, sales2012#x, earnings2013#x, sales2013#x, earnings2014#x, sales2014#x] + + +-- !query +SELECT * FROM courseEarningsAndSales +UNPIVOT ( + (earnings, sales) FOR year IN ((earnings2012, sales2012) as `2012`, (earnings2013, sales2013) as `2013`, (earnings2014, sales2014) as `2014`) +) +-- !query analysis +Project [course#x, year#x, earnings#x, sales#x] ++- Filter isnotnull(coalesce(earnings#x, sales#x)) + +- Expand [[course#x, 2012, earnings2012#x, sales2012#x], [course#x, 2013, earnings2013#x, sales2013#x], [course#x, 2014, earnings2014#x, sales2014#x]], [course#x, year#x, earnings#x, sales#x] + +- SubqueryAlias courseearningsandsales + +- View (`courseEarningsAndSales`, [course#x,earnings2012#x,sales2012#x,earnings2013#x,sales2013#x,earnings2014#x,sales2014#x]) + +- Project [cast(course#x as string) AS course#x, cast(earnings2012#x as int) AS earnings2012#x, cast(sales2012#x as int) AS sales2012#x, cast(earnings2013#x as int) AS earnings2013#x, cast(sales2013#x as int) AS sales2013#x, cast(earnings2014#x as int) AS earnings2014#x, cast(sales2014#x as int) AS sales2014#x] + +- Project [course#x, earnings2012#x, sales2012#x, earnings2013#x, sales2013#x, earnings2014#x, sales2014#x] + +- SubqueryAlias courseEarningsAndSales + +- LocalRelation [course#x, earnings2012#x, sales2012#x, earnings2013#x, sales2013#x, earnings2014#x, sales2014#x] 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 new file mode 100644 index 0000000000000..d93bbdb5400e2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/url-functions.sql.out @@ -0,0 +1,104 @@ +-- Automatically generated by SQLQueryTestSuite +-- !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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- 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] ++- OneRowRelation + + +-- !query +select url_encode('https://spark.apache.org') +-- !query analysis +Project [url_encode(https://spark.apache.org) AS url_encode(https://spark.apache.org)#x] ++- OneRowRelation + + +-- !query +select url_encode('inva lid://user:pass@host/file\\;param?query\\;p2') +-- !query analysis +Project [url_encode(inva lid://user:pass@host/file\;param?query\;p2) AS url_encode(inva lid://user:pass@host/file\;param?query\;p2)#x] ++- OneRowRelation + + +-- !query +select url_encode(null) +-- !query analysis +Project [url_encode(cast(null as string)) AS url_encode(NULL)#x] ++- OneRowRelation + + +-- !query +select url_decode('https%3A%2F%2Fspark.apache.org') +-- !query analysis +Project [url_decode(https%3A%2F%2Fspark.apache.org) AS url_decode(https%3A%2F%2Fspark.apache.org)#x] ++- OneRowRelation + + +-- !query +select url_decode('http%3A%2F%2spark.apache.org') +-- !query analysis +Project [url_decode(http%3A%2F%2spark.apache.org) AS url_decode(http%3A%2F%2spark.apache.org)#x] ++- OneRowRelation + + +-- !query +select url_decode('inva lid://user:pass@host/file\\;param?query\\;p2') +-- !query analysis +Project [url_decode(inva lid://user:pass@host/file\;param?query\;p2) AS url_decode(inva lid://user:pass@host/file\;param?query\;p2)#x] ++- OneRowRelation + + +-- !query +select url_decode(null) +-- !query analysis +Project [url_decode(cast(null as string)) AS url_decode(NULL)#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out new file mode 100644 index 0000000000000..8dc5a674897af --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out @@ -0,0 +1,808 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query analysis +CreateViewCommand `nt1`, select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1), false, false, LocalTempView, true + +- Project [k#x, v1#x] + +- SubqueryAlias nt1 + +- LocalRelation [k#x, v1#x] + + +-- !query +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5), + ("four", 4) + as nt2(k, v2) +-- !query analysis +CreateViewCommand `nt2`, select * from values + ("one", 1), + ("two", 22), + ("one", 5), + ("four", 4) + as nt2(k, v2), false, false, LocalTempView, true + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 left outer join nt2 using (k) +-- !query analysis +Project [k#x, v1#x, v2#x] ++- Project [k#x, v1#x, v2#x] + +- Join LeftOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k FROM nt1 left outer join nt2 using (k) +-- !query analysis +Project [k#x] ++- Project [k#x, v1#x, v2#x] + +- Join LeftOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.*, nt2.* FROM nt1 left outer join nt2 using (k) +-- !query analysis +Project [k#x, v1#x, k#x, v2#x] ++- Project [k#x, v1#x, v2#x, k#x] + +- Join LeftOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.k, nt2.k FROM nt1 left outer join nt2 using (k) +-- !query analysis +Project [k#x, k#x] ++- Project [k#x, v1#x, v2#x, k#x] + +- Join LeftOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k FROM (SELECT nt2.k FROM nt1 left outer join nt2 using (k)) +-- !query analysis +Project [k#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [k#x] + +- Project [k#x, v1#x, v2#x, k#x] + +- Join LeftOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt2.k AS key FROM nt1 left outer join nt2 using (k) ORDER BY key +-- !query analysis +Sort [key#x ASC NULLS FIRST], true ++- Project [k#x AS key#x] + +- Project [k#x, v1#x, v2#x, k#x] + +- Join LeftOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.k, nt2.k FROM nt1 left outer join nt2 using (k) ORDER BY nt2.k +-- !query analysis +Sort [k#x ASC NULLS FIRST], true ++- Project [k#x, k#x] + +- Project [k#x, v1#x, v2#x, k#x] + +- Join LeftOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k, nt1.k FROM nt1 left outer join nt2 using (k) +-- !query analysis +Project [k#x, k#x] ++- Project [k#x, v1#x, v2#x] + +- Join LeftOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k, nt2.k FROM nt1 left outer join nt2 using (k) +-- !query analysis +Project [k#x, k#x] ++- Project [k#x, v1#x, v2#x, k#x] + +- Join LeftOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 left semi join nt2 using (k) +-- !query analysis +Project [k#x, v1#x] ++- Project [k#x, v1#x] + +- Join LeftSemi, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k FROM nt1 left semi join nt2 using (k) +-- !query analysis +Project [k#x] ++- Project [k#x, v1#x] + +- Join LeftSemi, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.* FROM nt1 left semi join nt2 using (k) +-- !query analysis +Project [k#x, v1#x] ++- Project [k#x, v1#x] + +- Join LeftSemi, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.k FROM nt1 left semi join nt2 using (k) +-- !query analysis +Project [k#x] ++- Project [k#x, v1#x] + +- Join LeftSemi, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k, nt1.k FROM nt1 left semi join nt2 using (k) +-- !query analysis +Project [k#x, k#x] ++- Project [k#x, v1#x] + +- Join LeftSemi, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 right outer join nt2 using (k) +-- !query analysis +Project [k#x, v1#x, v2#x] ++- Project [k#x, v1#x, v2#x] + +- Join RightOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k FROM nt1 right outer join nt2 using (k) +-- !query analysis +Project [k#x] ++- Project [k#x, v1#x, v2#x] + +- Join RightOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.*, nt2.* FROM nt1 right outer join nt2 using (k) +-- !query analysis +Project [k#x, v1#x, k#x, v2#x] ++- Project [k#x, v1#x, v2#x, k#x] + +- Join RightOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.k, nt2.k FROM nt1 right outer join nt2 using (k) +-- !query analysis +Project [k#x, k#x] ++- Project [k#x, v1#x, v2#x, k#x] + +- Join RightOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k FROM (SELECT nt1.k FROM nt1 right outer join nt2 using (k)) +-- !query analysis +Project [k#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [k#x] + +- Project [k#x, v1#x, v2#x, k#x] + +- Join RightOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.k AS key FROM nt1 right outer join nt2 using (k) ORDER BY key +-- !query analysis +Sort [key#x ASC NULLS FIRST], true ++- Project [k#x AS key#x] + +- Project [k#x, v1#x, v2#x, k#x] + +- Join RightOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k, nt1.k FROM nt1 right outer join nt2 using (k) +-- !query analysis +Project [k#x, k#x] ++- Project [k#x, v1#x, v2#x, k#x] + +- Join RightOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k, nt2.k FROM nt1 right outer join nt2 using (k) +-- !query analysis +Project [k#x, k#x] ++- Project [k#x, v1#x, v2#x] + +- Join RightOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 full outer join nt2 using (k) +-- !query analysis +Project [k#x, v1#x, v2#x] ++- Project [coalesce(k#x, k#x) AS k#x, v1#x, v2#x] + +- Join FullOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k FROM nt1 full outer join nt2 using (k) +-- !query analysis +Project [k#x] ++- Project [coalesce(k#x, k#x) AS k#x, v1#x, v2#x] + +- Join FullOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.*, nt2.* FROM nt1 full outer join nt2 using (k) +-- !query analysis +Project [k#x, v1#x, k#x, v2#x] ++- Project [coalesce(k#x, k#x) AS k#x, v1#x, v2#x, k#x, k#x] + +- Join FullOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.k, nt2.k FROM nt1 full outer join nt2 using (k) +-- !query analysis +Project [k#x, k#x] ++- Project [coalesce(k#x, k#x) AS k#x, v1#x, v2#x, k#x, k#x] + +- Join FullOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k FROM (SELECT nt2.k FROM nt1 full outer join nt2 using (k)) +-- !query analysis +Project [k#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [k#x] + +- Project [coalesce(k#x, k#x) AS k#x, v1#x, v2#x, k#x, k#x] + +- Join FullOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt2.k AS key FROM nt1 full outer join nt2 using (k) ORDER BY key +-- !query analysis +Sort [key#x ASC NULLS FIRST], true ++- Project [k#x AS key#x] + +- Project [coalesce(k#x, k#x) AS k#x, v1#x, v2#x, k#x, k#x] + +- Join FullOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k, nt1.k FROM nt1 full outer join nt2 using (k) +-- !query analysis +Project [k#x, k#x] ++- Project [coalesce(k#x, k#x) AS k#x, v1#x, v2#x, k#x, k#x] + +- Join FullOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k, nt2.k FROM nt1 full outer join nt2 using (k) +-- !query analysis +Project [k#x, k#x] ++- Project [coalesce(k#x, k#x) AS k#x, v1#x, v2#x, k#x, k#x] + +- Join FullOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT * FROM nt1 full outer join nt2 using (k) +-- !query analysis +Project [k#x, v1#x, v2#x] ++- Project [coalesce(k#x, k#x) AS k#x, v1#x, v2#x] + +- Join FullOuter, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k FROM nt1 inner join nt2 using (k) +-- !query analysis +Project [k#x] ++- Project [k#x, v1#x, v2#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.*, nt2.* FROM nt1 inner join nt2 using (k) +-- !query analysis +Project [k#x, v1#x, k#x, v2#x] ++- Project [k#x, v1#x, v2#x, k#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt1.k, nt2.k FROM nt1 inner join nt2 using (k) +-- !query analysis +Project [k#x, k#x] ++- Project [k#x, v1#x, v2#x, k#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k FROM (SELECT nt2.k FROM nt1 inner join nt2 using (k)) +-- !query analysis +Project [k#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [k#x] + +- Project [k#x, v1#x, v2#x, k#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT nt2.k AS key FROM nt1 inner join nt2 using (k) ORDER BY key +-- !query analysis +Sort [key#x ASC NULLS FIRST], true ++- Project [k#x AS key#x] + +- Project [k#x, v1#x, v2#x, k#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k, nt1.k FROM nt1 inner join nt2 using (k) +-- !query analysis +Project [k#x, k#x] ++- Project [k#x, v1#x, v2#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] + + +-- !query +SELECT k, nt2.k FROM nt1 inner join nt2 using (k) +-- !query analysis +Project [k#x, k#x] ++- Project [k#x, v1#x, v2#x, k#x] + +- Join Inner, (k#x = k#x) + :- SubqueryAlias nt1 + : +- View (`nt1`, [k#x,v1#x]) + : +- Project [cast(k#x as string) AS k#x, cast(v1#x as int) AS v1#x] + : +- Project [k#x, v1#x] + : +- SubqueryAlias nt1 + : +- LocalRelation [k#x, v1#x] + +- SubqueryAlias nt2 + +- View (`nt2`, [k#x,v2#x]) + +- Project [cast(k#x as string) AS k#x, cast(v2#x as int) AS v2#x] + +- Project [k#x, v2#x] + +- SubqueryAlias nt2 + +- LocalRelation [k#x, v2#x] 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 new file mode 100644 index 0000000000000..454880cb7550b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out @@ -0,0 +1,1281 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), +(null, null, null, null, null, null), +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) +AS testData(val, val_long, val_double, val_date, val_timestamp, cate) +-- !query analysis +CreateViewCommand `testData`, SELECT * FROM VALUES +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), +(null, null, null, null, null, null), +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) +AS testData(val, val_long, val_double, val_date, val_timestamp, cate), false, true, LocalTempView, true + +- 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 +CREATE OR REPLACE TEMPORARY VIEW basic_pays AS SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary) +-- !query analysis +CreateViewCommand `basic_pays`, SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary), false, true, LocalTempView, true + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- LocalRelation [employee_name#x, department#x, salary#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW test_ignore_null AS SELECT * FROM VALUES +('a', 0, null), +('a', 1, 'x'), +('b', 2, null), +('c', 3, null), +('a', 4, 'y'), +('b', 5, null), +('a', 6, 'z'), +('a', 7, 'v'), +('a', 8, null) +AS test_ignore_null(content, id, v) +-- !query analysis +CreateViewCommand `test_ignore_null`, SELECT * FROM VALUES +('a', 0, null), +('a', 1, 'x'), +('b', 2, null), +('c', 3, null), +('a', 4, 'y'), +('b', 5, null), +('a', 6, 'z'), +('a', 7, 'v'), +('a', 8, null) +AS test_ignore_null(content, id, v), false, true, LocalTempView, true + +- Project [content#x, id#x, v#x] + +- SubqueryAlias test_ignore_null + +- LocalRelation [content#x, id#x, v#x] + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) FROM testData +ORDER BY cate, val +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true ++- Project [val#x, cate#x, count(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND CURRENT ROW)#xL] + +- Project [val#x, cate#x, count(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND CURRENT ROW)#xL, count(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND CURRENT ROW)#xL] + +- Window [count(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, currentrow$(), currentrow$())) AS count(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST ROWS BETWEEN CURRENT ROW AND CURRENT ROW)#xL], [cate#x], [val#x ASC NULLS FIRST] + +- Project [val#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 val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val +ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true ++- Project [val#x, cate#x, sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)#xL] + +- Project [val#x, cate#x, sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)#xL, sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)#xL] + +- Window [sum(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), 1)) AS sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)#xL], [cate#x], [val#x ASC NULLS FIRST] + +- Project [val#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 val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long +ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "expectedType" : "\"INT\"", + "exprType" : "\"BIGINT\"", + "location" : "upper", + "sqlExpr" : "\"ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 42, + "stopIndex" : 128, + "fragment" : "(PARTITION BY cate ORDER BY val_long\nROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE 1 PRECEDING) FROM testData +ORDER BY cate, val +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true ++- Project [val#x, cate#x, count(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND CURRENT ROW)#xL] + +- Project [val#x, cate#x, count(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND CURRENT ROW)#xL, count(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND CURRENT ROW)#xL] + +- Window [count(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -1, currentrow$())) AS count(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND CURRENT ROW)#xL], [cate#x], [val#x ASC NULLS FIRST] + +- Project [val#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 val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true ++- Project [val#x, cate#x, sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] + +- Project [val#x, cate#x, sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] + +- Window [sum(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), 1)) AS sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL], [cate#x], [val#x ASC NULLS FIRST] + +- Project [val#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 val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long +RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val_long#xL ASC NULLS FIRST], true ++- Project [val_long#xL, cate#x, sum(val_long) OVER (PARTITION BY cate ORDER BY val_long ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING)#xL] + +- Project [val_long#xL, cate#x, sum(val_long) OVER (PARTITION BY cate ORDER BY val_long ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING)#xL, sum(val_long) OVER (PARTITION BY cate ORDER BY val_long ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING)#xL] + +- Window [sum(val_long#xL) windowspecdefinition(cate#x, val_long#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), 2147483648)) AS sum(val_long) OVER (PARTITION BY cate ORDER BY val_long ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING)#xL], [cate#x], [val_long#xL ASC NULLS FIRST] + +- Project [val_long#xL, 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 val_double, cate, sum(val_double) OVER(PARTITION BY cate ORDER BY val_double +RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY cate, val_double +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val_double#x ASC NULLS FIRST], true ++- Project [val_double#x, cate#x, sum(val_double) OVER (PARTITION BY cate ORDER BY val_double ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING)#x] + +- Project [val_double#x, cate#x, sum(val_double) OVER (PARTITION BY cate ORDER BY val_double ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING)#x, sum(val_double) OVER (PARTITION BY cate ORDER BY val_double ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING)#x] + +- Window [sum(val_double#x) windowspecdefinition(cate#x, val_double#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), cast(2.5 as double))) AS sum(val_double) OVER (PARTITION BY cate ORDER BY val_double ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING)#x], [cate#x], [val_double#x ASC NULLS FIRST] + +- Project [val_double#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 val_date, cate, max(val_date) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY cate, val_date +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val_date#x ASC NULLS FIRST], true ++- Project [val_date#x, cate#x, max(val_date) OVER (PARTITION BY cate ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING)#x] + +- Project [val_date#x, cate#x, max(val_date) OVER (PARTITION BY cate ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING)#x, max(val_date) OVER (PARTITION BY cate ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING)#x] + +- Window [max(val_date#x) windowspecdefinition(cate#x, val_date#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), 2)) AS max(val_date) OVER (PARTITION BY cate ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING)#x], [cate#x], [val_date#x ASC NULLS FIRST] + +- Project [val_date#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 val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY cate, val_timestamp +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val_timestamp#x ASC NULLS FIRST], true ++- Project [val_timestamp#x, cate#x, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY 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, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY 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 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, val_timestamp#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), INTERVAL '23 04' DAY TO HOUR)) AS avg(val_timestamp) OVER (PARTITION BY cate ORDER BY val_timestamp ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '23 04' DAY TO HOUR FOLLOWING)#x], [cate#x], [val_timestamp#x ASC NULLS FIRST] + +- Project [val_timestamp#x, cate#x, cast(val_timestamp#x as double) AS _w0#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 val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) +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 ++- 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] + +- 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 val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval '1-1' year to month FOLLOWING) FROM testData +ORDER BY cate, val_timestamp +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val_timestamp#x ASC NULLS FIRST], true ++- Project [val_timestamp#x, cate#x, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY 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, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY 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 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, val_timestamp#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), INTERVAL '1-1' YEAR TO MONTH)) AS avg(val_timestamp) OVER (PARTITION BY cate ORDER BY val_timestamp ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1-1' YEAR TO MONTH FOLLOWING)#x], [cate#x], [val_timestamp#x ASC NULLS FIRST] + +- Project [val_timestamp#x, cate#x, cast(val_timestamp#x as double) AS _w0#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 val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) +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 ++- 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] + +- 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 val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData +ORDER BY cate, val_timestamp +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val_timestamp#x ASC NULLS FIRST], true ++- Project [val_timestamp#x, cate#x, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY 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, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY 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 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, val_timestamp#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 val_timestamp ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1 02:03:04.001' DAY TO SECOND FOLLOWING)#x], [cate#x], [val_timestamp#x ASC NULLS FIRST] + +- Project [val_timestamp#x, cate#x, cast(val_timestamp#x as double) AS _w0#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 val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) +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 ++- 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] + +- 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 val_date, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND interval '1-1' year to month FOLLOWING) FROM testData +ORDER BY cate, val_date +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val_date#x ASC NULLS FIRST], true ++- Project [val_date#x, cate#x, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1-1' YEAR TO MONTH FOLLOWING)#x] + +- Project [val_date#x, cate#x, _w0#x, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY val_date 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 val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1-1' YEAR TO MONTH FOLLOWING)#x] + +- Window [avg(_w0#x) windowspecdefinition(cate#x, val_date#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), INTERVAL '1-1' YEAR TO MONTH)) AS avg(val_timestamp) OVER (PARTITION BY cate ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1-1' YEAR TO MONTH FOLLOWING)#x], [cate#x], [val_date#x ASC NULLS FIRST] + +- Project [val_date#x, cate#x, cast(val_timestamp#x as double) AS _w0#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 val_date, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData +ORDER BY cate, val_date +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpecType" : "\"DATE\"", + "sqlExpr" : "\"(PARTITION BY cate ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1 02:03:04.001' DAY TO SECOND FOLLOWING)\"", + "valueBoundaryType" : "\"INTERVAL DAY TO SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 47, + "stopIndex" : 160, + "fragment" : "(PARTITION BY cate ORDER BY val_date\nRANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val DESC +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true ++- Project [val#x, cate#x, sum(val) OVER (PARTITION BY cate ORDER BY val DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] + +- Project [val#x, cate#x, sum(val) OVER (PARTITION BY cate ORDER BY val DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL, sum(val) OVER (PARTITION BY cate ORDER BY val DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL] + +- Window [sum(val#x) windowspecdefinition(cate#x, val#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), 1)) AS sum(val) OVER (PARTITION BY cate ORDER BY val DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)#xL], [cate#x], [val#x DESC NULLS LAST] + +- Project [val#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 val, cate, count(val) OVER(PARTITION BY cate +ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_INVALID_BOUND", + "sqlState" : "42K09", + "messageParameters" : { + "lower" : "\"UNBOUNDED FOLLOWING\"", + "sqlExpr" : "\"ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING\"", + "upper" : "\"1\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 101, + "fragment" : "(PARTITION BY cate\nROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"(PARTITION BY cate RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 94, + "fragment" : "(PARTITION BY cate\nRANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val, cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_MULTI_ORDER", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpec" : "val#x ASC NULLS FIRST,cate#x ASC NULLS FIRST", + "sqlExpr" : "\"(PARTITION BY cate ORDER BY val ASC NULLS FIRST, cate ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 113, + "fragment" : "(PARTITION BY cate ORDER BY val, cate\nRANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY current_timestamp +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpecType" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(PARTITION BY cate ORDER BY current_timestamp() ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"", + "valueBoundaryType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 121, + "fragment" : "(PARTITION BY cate ORDER BY current_timestamp\nRANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY cate, val +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_WRONG_COMPARISON", + "sqlState" : "42K09", + "messageParameters" : { + "comparison" : "less than or equal", + "sqlExpr" : "\"RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 107, + "fragment" : "(PARTITION BY cate ORDER BY val\nRANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Frame bound value must be a literal." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 96, + "stopIndex" : 117, + "fragment" : "current_date PRECEDING" + } ] +} + + +-- !query +SELECT val, cate, +max(val) OVER w AS max, +min(val) OVER w AS min, +min(val) OVER w AS min, +count(val) OVER w AS count, +sum(val) OVER w AS sum, +avg(val) OVER w AS avg, +stddev(val) OVER w AS stddev, +first_value(val) OVER w AS first_value, +first_value(val, true) OVER w AS first_value_ignore_null, +first_value(val, false) OVER w AS first_value_contain_null, +any_value(val) OVER w AS any_value, +any_value(val, true) OVER w AS any_value_ignore_null, +any_value(val, false) OVER w AS any_value_contain_null, +last_value(val) OVER w AS last_value, +last_value(val, true) OVER w AS last_value_ignore_null, +last_value(val, false) OVER w AS last_value_contain_null, +rank() OVER w AS rank, +dense_rank() OVER w AS dense_rank, +cume_dist() OVER w AS cume_dist, +percent_rank() OVER w AS percent_rank, +ntile(2) OVER w AS ntile, +row_number() OVER w AS row_number, +var_pop(val) OVER w AS var_pop, +var_samp(val) OVER w AS var_samp, +approx_count_distinct(val) OVER w AS approx_count_distinct, +covar_pop(val, val_long) OVER w AS covar_pop, +corr(val, val_long) OVER w AS corr, +stddev_samp(val) OVER w AS stddev_samp, +stddev_pop(val) OVER w AS stddev_pop, +collect_list(val) OVER w AS collect_list, +collect_set(val) OVER w AS collect_set, +skewness(val_double) OVER w AS skewness, +kurtosis(val_double) OVER w AS kurtosis +FROM testData +WINDOW w AS (PARTITION BY cate ORDER BY val) +ORDER BY cate, val +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true ++- Project [val#x, cate#x, max#x, min#x, min#x, count#xL, sum#xL, avg#x, stddev#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, rank#x, dense_rank#x, cume_dist#x, percent_rank#x, ntile#x, row_number#x, ... 11 more fields] + +- Project [val#x, cate#x, _w0#x, _w1#x, val_double#x, max#x, min#x, min#x, count#xL, sum#xL, avg#x, stddev#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, rank#x, dense_rank#x, cume_dist#x, ... 47 more fields] + +- Window [max(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS max#x, min(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS min#x, min(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS min#x, count(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS count#xL, sum(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS sum#xL, avg(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS avg#x, stddev(_w0#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS stddev#x, first_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value#x, first_value(val#x, true) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value_ignore_null#x, first_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value_contain_null#x, any_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value#x, any_value(val#x, true) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value_ignore_null#x, any_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value_contain_null#x, last_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value#x, last_value(val#x, true) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value_ignore_null#x, last_value(val#x, false) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value_contain_null#x, rank(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#x, dense_rank(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS dense_rank#x, cume_dist() windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS cume_dist#x, percent_rank(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS percent_rank#x, ntile(2) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ntile#x, row_number() windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS row_number#x, var_pop(_w0#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS var_pop#x, var_samp(_w0#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS var_samp#x, ... 9 more fields], [cate#x], [val#x ASC NULLS FIRST] + +- Project [val#x, cate#x, cast(val#x as double) AS _w0#x, cast(val_long#xL as double) AS _w1#x, val_double#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 val, cate, avg(null) OVER(PARTITION BY cate ORDER BY val) FROM testData ORDER BY cate, val +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true ++- Project [val#x, cate#x, avg(NULL) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Project [val#x, cate#x, avg(NULL) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x, avg(NULL) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x] + +- Window [avg(cast(null as double)) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS avg(NULL) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [cate#x], [val#x ASC NULLS FIRST] + +- Project [val#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 val, cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY cate, val +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1037", + "messageParameters" : { + "wf" : "row_number()" + } +} + + +-- !query +SELECT val, cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true ++- Project [val#x, cate#x, sum(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, avg(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Project [val#x, cate#x, sum(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, avg(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, sum(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, avg(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Window [sum(val#x) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS sum(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#xL, avg(val#x) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg(val) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] + +- Project [val#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 val, cate, +first_value(false) OVER w AS first_value, +first_value(true, true) OVER w AS first_value_ignore_null, +first_value(false, false) OVER w AS first_value_contain_null, +any_value(false) OVER w AS any_value, +any_value(true, true) OVER w AS any_value_ignore_null, +any_value(false, false) OVER w AS any_value_contain_null, +last_value(false) OVER w AS last_value, +last_value(true, true) OVER w AS last_value_ignore_null, +last_value(false, false) OVER w AS last_value_contain_null +FROM testData +WINDOW w AS () +ORDER BY cate, val +-- !query analysis +Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true ++- Project [val#x, cate#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x] + +- Project [val#x, cate#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x, first_value#x, first_value_ignore_null#x, first_value_contain_null#x, any_value#x, any_value_ignore_null#x, any_value_contain_null#x, last_value#x, last_value_ignore_null#x, last_value_contain_null#x] + +- Window [first_value(false, false) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS first_value#x, first_value(true, true) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS first_value_ignore_null#x, first_value(false, false) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS first_value_contain_null#x, any_value(false, false) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS any_value#x, any_value(true, true) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS any_value_ignore_null#x, any_value(false, false) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS any_value_contain_null#x, last_value(false, false) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS last_value#x, last_value(true, true) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS last_value_ignore_null#x, last_value(false, false) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS last_value_contain_null#x] + +- Project [val#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 cate, sum(val) OVER (w) +FROM testData +WHERE val is not null +WINDOW w AS (PARTITION BY cate ORDER BY val) +-- !query analysis +Project [cate#x, sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] ++- Project [cate#x, val#x, sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL, sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL] + +- Window [sum(val#x) windowspecdefinition(cate#x, val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS sum(val) OVER (PARTITION BY cate ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#xL], [cate#x], [val#x ASC NULLS FIRST] + +- Project [cate#x, val#x] + +- Filter isnotnull(val#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 val, cate, +count(val) FILTER (WHERE val > 1) OVER(PARTITION BY cate) +FROM testData ORDER BY cate, val +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1030" +} + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC) +ORDER BY salary DESC +-- !query analysis +Sort [salary#x DESC NULLS LAST], true ++- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Window [first_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS highest_salary#x, any_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS highest_salary#x, nth_value(employee_name#x, 2, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS second_highest_salary#x], [salary#x DESC NULLS LAST] + +- Project [employee_name#x, salary#x] + +- SubqueryAlias basic_pays + +- View (`basic_pays`, [employee_name#x,department#x,salary#x]) + +- Project [cast(employee_name#x as string) AS employee_name#x, cast(department#x as string) AS department#x, cast(salary#x as int) AS salary#x] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- LocalRelation [employee_name#x, department#x, salary#x] + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY salary DESC +-- !query analysis +Sort [salary#x DESC NULLS LAST], true ++- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Window [first_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS highest_salary#x, any_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS highest_salary#x, nth_value(employee_name#x, 2, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS second_highest_salary#x], [salary#x DESC NULLS LAST] + +- Project [employee_name#x, salary#x] + +- SubqueryAlias basic_pays + +- View (`basic_pays`, [employee_name#x,department#x,salary#x]) + +- Project [cast(employee_name#x as string) AS employee_name#x, cast(department#x as string) AS department#x, cast(salary#x as int) AS salary#x] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- LocalRelation [employee_name#x, department#x, salary#x] + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY salary DESC +-- !query analysis +Sort [salary#x DESC NULLS LAST], true ++- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Window [first_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS highest_salary#x, any_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS highest_salary#x, nth_value(employee_name#x, 2, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS second_highest_salary#x], [salary#x DESC NULLS LAST] + +- Project [employee_name#x, salary#x] + +- SubqueryAlias basic_pays + +- View (`basic_pays`, [employee_name#x,department#x,salary#x]) + +- Project [cast(employee_name#x as string) AS employee_name#x, cast(department#x as string) AS department#x, cast(salary#x as int) AS salary#x] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- LocalRelation [employee_name#x, department#x, salary#x] + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary RANGE BETWEEN 2000 PRECEDING AND 1000 FOLLOWING) +ORDER BY salary +-- !query analysis +Sort [salary#x ASC NULLS FIRST], true ++- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Window [first_value(employee_name#x, false) windowspecdefinition(salary#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -2000, 1000)) AS highest_salary#x, any_value(employee_name#x, false) windowspecdefinition(salary#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -2000, 1000)) AS highest_salary#x, nth_value(employee_name#x, 2, false) windowspecdefinition(salary#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -2000, 1000)) AS second_highest_salary#x], [salary#x ASC NULLS FIRST] + +- Project [employee_name#x, salary#x] + +- SubqueryAlias basic_pays + +- View (`basic_pays`, [employee_name#x,department#x,salary#x]) + +- Project [cast(employee_name#x as string) AS employee_name#x, cast(department#x as string) AS department#x, cast(salary#x as int) AS salary#x] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- LocalRelation [employee_name#x, department#x, salary#x] + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING) +ORDER BY salary DESC +-- !query analysis +Sort [salary#x DESC NULLS LAST], true ++- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Window [first_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RowFrame, -2, 2)) AS highest_salary#x, any_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RowFrame, -2, 2)) AS highest_salary#x, nth_value(employee_name#x, 2, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RowFrame, -2, 2)) AS second_highest_salary#x], [salary#x DESC NULLS LAST] + +- Project [employee_name#x, salary#x] + +- SubqueryAlias basic_pays + +- View (`basic_pays`, [employee_name#x,department#x,salary#x]) + +- Project [cast(employee_name#x as string) AS employee_name#x, cast(department#x as string) AS department#x, cast(salary#x as int) AS salary#x] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- LocalRelation [employee_name#x, department#x, salary#x] + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +ORDER BY salary DESC +-- !query analysis +Sort [salary#x DESC NULLS LAST], true ++- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Window [first_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), unboundedfollowing$())) AS highest_salary#x, any_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), unboundedfollowing$())) AS highest_salary#x, nth_value(employee_name#x, 2, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, currentrow$(), unboundedfollowing$())) AS second_highest_salary#x], [salary#x DESC NULLS LAST] + +- Project [employee_name#x, salary#x] + +- SubqueryAlias basic_pays + +- View (`basic_pays`, [employee_name#x,department#x,salary#x]) + +- Project [cast(employee_name#x as string) AS employee_name#x, cast(department#x as string) AS department#x, cast(salary#x as int) AS salary#x] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- LocalRelation [employee_name#x, department#x, salary#x] + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY salary DESC +-- !query analysis +Sort [salary#x DESC NULLS LAST], true ++- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Window [first_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), unboundedfollowing$())) AS highest_salary#x, any_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), unboundedfollowing$())) AS highest_salary#x, nth_value(employee_name#x, 2, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), unboundedfollowing$())) AS second_highest_salary#x], [salary#x DESC NULLS LAST] + +- Project [employee_name#x, salary#x] + +- SubqueryAlias basic_pays + +- View (`basic_pays`, [employee_name#x,department#x,salary#x]) + +- Project [cast(employee_name#x as string) AS employee_name#x, cast(department#x as string) AS department#x, cast(salary#x as int) AS salary#x] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- LocalRelation [employee_name#x, department#x, salary#x] + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY salary DESC +-- !query analysis +Sort [salary#x DESC NULLS LAST], true ++- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Window [first_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS highest_salary#x, any_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS highest_salary#x, nth_value(employee_name#x, 2, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS second_highest_salary#x], [salary#x DESC NULLS LAST] + +- Project [employee_name#x, salary#x] + +- SubqueryAlias basic_pays + +- View (`basic_pays`, [employee_name#x,department#x,salary#x]) + +- Project [cast(employee_name#x as string) AS employee_name#x, cast(department#x as string) AS department#x, cast(salary#x as int) AS salary#x] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- LocalRelation [employee_name#x, department#x, salary#x] + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary DESC +-- !query analysis +Sort [salary#x DESC NULLS LAST], true ++- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Project [employee_name#x, salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x, highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Window [first_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), 1)) AS highest_salary#x, any_value(employee_name#x, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), 1)) AS highest_salary#x, nth_value(employee_name#x, 2, false) windowspecdefinition(salary#x DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), 1)) AS second_highest_salary#x], [salary#x DESC NULLS LAST] + +- Project [employee_name#x, salary#x] + +- SubqueryAlias basic_pays + +- View (`basic_pays`, [employee_name#x,department#x,salary#x]) + +- Project [cast(employee_name#x as string) AS employee_name#x, cast(department#x as string) AS department#x, cast(salary#x as int) AS salary#x] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- LocalRelation [employee_name#x, department#x, salary#x] + + +-- !query +SELECT + employee_name, + department, + salary, + FIRST_VALUE(employee_name) OVER w highest_salary, + NTH_VALUE(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS ( + PARTITION BY department + ORDER BY salary DESC + RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING +) +ORDER BY department +-- !query analysis +Sort [department#x ASC NULLS FIRST], true ++- Project [employee_name#x, department#x, salary#x, highest_salary#x, second_highest_salary#x] + +- Project [employee_name#x, department#x, salary#x, highest_salary#x, second_highest_salary#x, highest_salary#x, second_highest_salary#x] + +- Window [first_value(employee_name#x, false) windowspecdefinition(department#x, salary#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), unboundedfollowing$())) AS highest_salary#x, nth_value(employee_name#x, 2, false) windowspecdefinition(department#x, salary#x DESC NULLS LAST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), unboundedfollowing$())) AS second_highest_salary#x], [department#x], [salary#x DESC NULLS LAST] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- View (`basic_pays`, [employee_name#x,department#x,salary#x]) + +- Project [cast(employee_name#x as string) AS employee_name#x, cast(department#x as string) AS department#x, cast(salary#x as int) AS salary#x] + +- Project [employee_name#x, department#x, salary#x] + +- SubqueryAlias basic_pays + +- LocalRelation [employee_name#x, department#x, salary#x] + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW + w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), + w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 2 FOLLOWING) +ORDER BY salary DESC +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX", + "sqlState" : "42000", + "messageParameters" : { + "inputString" : "The definition of window `w` is repetitive." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 226, + "stopIndex" : 394, + "fragment" : "WINDOW\n w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING),\n w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 2 FOLLOWING)" + } ] +} + + +-- !query +SELECT + content, + id, + v, + lead(v, 0) IGNORE NULLS OVER w lead_0, + lead(v, 1) IGNORE NULLS OVER w lead_1, + lead(v, 2) IGNORE NULLS OVER w lead_2, + lead(v, 3) IGNORE NULLS OVER w lead_3, + lag(v, 0) IGNORE NULLS OVER w lag_0, + 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, + 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, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +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] + +- SubqueryAlias test_ignore_null + +- View (`test_ignore_null`, [content#x,id#x,v#x]) + +- Project [cast(content#x as string) AS content#x, cast(id#x as int) AS id#x, cast(v#x as string) AS v#x] + +- Project [content#x, id#x, v#x] + +- SubqueryAlias test_ignore_null + +- LocalRelation [content#x, id#x, v#x] + + +-- !query +SELECT + content, + id, + v, + 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, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY id +-- !query analysis +Sort [id#x ASC NULLS FIRST], true ++- Project [content#x, id#x, v#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, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x] + +- Window [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]) + +- Project [cast(content#x as string) AS content#x, cast(id#x as int) AS id#x, cast(v#x as string) AS v#x] + +- Project [content#x, id#x, v#x] + +- SubqueryAlias test_ignore_null + +- LocalRelation [content#x, id#x, v#x] + + +-- !query +SELECT + content, + id, + v, + 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, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY id +-- !query analysis +Sort [id#x ASC NULLS FIRST], true ++- Project [content#x, id#x, v#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, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x] + +- Window [nth_value(v#x, 1, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS nth_value_1#x, nth_value(v#x, 2, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS nth_value_2#x, nth_value(v#x, 3, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS nth_value_3#x, first(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS first_value#x, any_value(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS any_value#x, last(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 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]) + +- Project [cast(content#x as string) AS content#x, cast(id#x as int) AS id#x, cast(v#x as string) AS v#x] + +- Project [content#x, id#x, v#x] + +- SubqueryAlias test_ignore_null + +- LocalRelation [content#x, id#x, v#x] + + +-- !query +SELECT + content, + id, + v, + 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, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN 2 PRECEDING AND 2 FOLLOWING) +ORDER BY id +-- !query analysis +Sort [id#x ASC NULLS FIRST], true ++- Project [content#x, id#x, v#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, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x] + +- Window [nth_value(v#x, 1, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -2, 2)) AS nth_value_1#x, nth_value(v#x, 2, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -2, 2)) AS nth_value_2#x, nth_value(v#x, 3, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -2, 2)) AS nth_value_3#x, first(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -2, 2)) AS first_value#x, any_value(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -2, 2)) AS any_value#x, last(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -2, 2)) 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]) + +- Project [cast(content#x as string) AS content#x, cast(id#x as int) AS id#x, cast(v#x as string) AS v#x] + +- Project [content#x, id#x, v#x] + +- SubqueryAlias test_ignore_null + +- LocalRelation [content#x, id#x, v#x] + + +-- !query +SELECT + content, + id, + v, + 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, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING) +ORDER BY id +-- !query analysis +Sort [id#x ASC NULLS FIRST], true ++- Project [content#x, id#x, v#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, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x] + +- Window [nth_value(v#x, 1, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, 2)) AS nth_value_1#x, nth_value(v#x, 2, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, 2)) AS nth_value_2#x, nth_value(v#x, 3, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, 2)) AS nth_value_3#x, first(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, 2)) AS first_value#x, any_value(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, 2)) AS any_value#x, last(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, 2)) 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]) + +- Project [cast(content#x as string) AS content#x, cast(id#x as int) AS id#x, cast(v#x as string) AS v#x] + +- Project [content#x, id#x, v#x] + +- SubqueryAlias test_ignore_null + +- LocalRelation [content#x, id#x, v#x] + + +-- !query +SELECT + content, + id, + v, + 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, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +ORDER BY id +-- !query analysis +Sort [id#x ASC NULLS FIRST], true ++- Project [content#x, id#x, v#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, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x] + +- Window [nth_value(v#x, 1, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), unboundedfollowing$())) AS nth_value_1#x, nth_value(v#x, 2, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), unboundedfollowing$())) AS nth_value_2#x, nth_value(v#x, 3, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), unboundedfollowing$())) AS nth_value_3#x, first(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), unboundedfollowing$())) AS first_value#x, any_value(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), unboundedfollowing$())) AS any_value#x, last(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), unboundedfollowing$())) 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]) + +- Project [cast(content#x as string) AS content#x, cast(id#x as int) AS id#x, cast(v#x as string) AS v#x] + +- Project [content#x, id#x, v#x] + +- SubqueryAlias test_ignore_null + +- LocalRelation [content#x, id#x, v#x] + + +-- !query +SELECT + content, + id, + v, + 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, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY id +-- !query analysis +Sort [id#x ASC NULLS FIRST], true ++- Project [content#x, id#x, v#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, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x] + +- Window [nth_value(v#x, 1, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), unboundedfollowing$())) AS nth_value_1#x, nth_value(v#x, 2, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), unboundedfollowing$())) AS nth_value_2#x, nth_value(v#x, 3, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), unboundedfollowing$())) AS nth_value_3#x, first(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), unboundedfollowing$())) AS first_value#x, any_value(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), unboundedfollowing$())) AS any_value#x, last(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), unboundedfollowing$())) 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]) + +- Project [cast(content#x as string) AS content#x, cast(id#x as int) AS id#x, cast(v#x as string) AS v#x] + +- Project [content#x, id#x, v#x] + +- SubqueryAlias test_ignore_null + +- LocalRelation [content#x, id#x, v#x] + + +-- !query +SELECT + content, + id, + v, + 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, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY id +-- !query analysis +Sort [id#x ASC NULLS FIRST], true ++- Project [content#x, id#x, v#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, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x] + +- Window [nth_value(v#x, 1, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS nth_value_1#x, nth_value(v#x, 2, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS nth_value_2#x, nth_value(v#x, 3, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS nth_value_3#x, first(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS first_value#x, any_value(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS any_value#x, last(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) 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]) + +- Project [cast(content#x as string) AS content#x, cast(id#x as int) AS id#x, cast(v#x as string) AS v#x] + +- Project [content#x, id#x, v#x] + +- SubqueryAlias test_ignore_null + +- LocalRelation [content#x, id#x, v#x] + + +-- !query +SELECT + content, + id, + v, + 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, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY id +-- !query analysis +Sort [id#x ASC NULLS FIRST], true ++- Project [content#x, id#x, v#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, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x] + +- Window [nth_value(v#x, 1, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), 1)) AS nth_value_1#x, nth_value(v#x, 2, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), 1)) AS nth_value_2#x, nth_value(v#x, 3, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), 1)) AS nth_value_3#x, first(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), 1)) AS first_value#x, any_value(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), 1)) AS any_value#x, last(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), 1)) 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]) + +- Project [cast(content#x as string) AS content#x, cast(id#x as int) AS id#x, cast(v#x as string) AS v#x] + +- Project [content#x, id#x, v#x] + +- SubqueryAlias test_ignore_null + +- LocalRelation [content#x, id#x, v#x] + + +-- !query +SELECT + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1004", + "messageParameters" : { + "windowName" : "w" + } +} + + +-- !query +SELECT + SUM(salary) OVER w sum_salary +FROM + basic_pays +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1004", + "messageParameters" : { + "windowName" : "w" + } +} 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 10ef2a32accc9..0760328e4f8c8 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 @@ -23,12 +23,13 @@ import org.apache.spark.{SparkException, SparkThrowable} import org.apache.spark.ErrorMessageFormat.MINIMAL import org.apache.spark.SparkThrowableHelper.getMessage import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.expressions.{CurrentDate, CurrentTimestampLike, CurrentUser, Literal} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.HiveResult.hiveResultString import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeCommandBase} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{DateType, StructType, TimestampType} trait SQLQueryTestHelper extends Logging { @@ -49,20 +50,51 @@ trait SQLQueryTestHelper extends Logging { .replaceAll("Created Time.*", s"Created Time $notIncludedMsg") .replaceAll("Last Access.*", s"Last Access $notIncludedMsg") .replaceAll("Partition Statistics\t\\d+", s"Partition Statistics\t$notIncludedMsg") + .replaceAll("CTERelationDef \\d+,", s"CTERelationDef xxxx,") + .replaceAll("CTERelationRef \\d+,", s"CTERelationRef xxxx,") + .replaceAll("@\\w*,", s"@xxxxxxxx,") .replaceAll("\\*\\(\\d+\\) ", "*") // remove the WholeStageCodegen codegenStageIds } - /** * Analyzes a query and returns the result as (schema of the output, normalized resolved plan * tree string representation). */ protected def getNormalizedQueryAnalysisResult( session: SparkSession, sql: String): (String, Seq[String]) = { + // Note that creating the following DataFrame includes eager execution for commands that create + // objects such as views. Therefore any following queries that reference these objects should + // find them in the catalog. val df = session.sql(sql) val schema = df.schema.catalogString - // Get the answer, but also get rid of the #1234 expression IDs that show up in analyzer plans. - (schema, Seq(replaceNotIncludedMsg(df.queryExecution.analyzed.toString))) + val analyzed = df.queryExecution.analyzed + // Determine if the analyzed plan contains any nondeterministic expressions. + var deterministic = true + analyzed.transformAllExpressionsWithSubqueries { + case expr: CurrentDate => + deterministic = false + expr + case expr: CurrentTimestampLike => + deterministic = false + expr + case expr: CurrentUser => + deterministic = false + expr + case expr: Literal if expr.dataType == DateType || expr.dataType == TimestampType => + deterministic = false + expr + case expr if !expr.deterministic => + deterministic = false + expr + } + if (deterministic) { + // Perform query analysis, but also get rid of the #1234 expression IDs that show up in the + // resolved plans. + (schema, Seq(replaceNotIncludedMsg(analyzed.toString))) + } else { + // The analyzed plan is nondeterministic so elide it from the result to keep tests reliable. + (schema, Seq("[Analyzer test output redacted due to nondeterminism]")) + } } /** Executes a query and returns the result as (schema of the output, normalized output). */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 5b6d587cb8df6..ce437dfb0ade7 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 @@ -127,8 +127,6 @@ import org.apache.spark.util.Utils * data from executing the query end-to-end. * * Each case has a golden result file in "spark/sql/core/src/test/resources/sql-tests/analyzer-results". - * Only input filenames in the "analyzerTestCaseList" below are included for this type of testing. - * In the future, we may expand the coverage to all of the input test files instead. */ // scalastyle:on line.size.limit @ExtendedSQLTest @@ -170,14 +168,12 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper // Create all the test cases. listTestCases.foreach(createScalaTestCase) - /** List of test cases to perform analyzer tests for. */ - protected def analyzerTestCaseList = Seq("array.sql") - /** A test case. */ protected trait TestCase { val name: String val inputFile: String val resultFile: String + def asAnalyzerTest(newName: String, newResultFile: String): TestCase } /** @@ -190,7 +186,9 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper protected trait AnsiTest /** Trait that indicates an analyzer test that shows the analyzed plan string as output. */ - protected trait AnalyzerTest + protected trait AnalyzerTest extends TestCase { + override def asAnalyzerTest(newName: String, newResultFile: String): AnalyzerTest = this + } /** Trait that indicates the default timestamp type is TimestampNTZType. */ protected trait TimestampNTZTest @@ -201,11 +199,17 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper /** A regular test case. */ protected case class RegularTestCase( - name: String, inputFile: String, resultFile: String) extends TestCase + name: String, inputFile: String, resultFile: String) extends TestCase { + override def asAnalyzerTest(newName: String, newResultFile: String): TestCase = + RegularAnalyzerTestCase(newName, inputFile, newResultFile) + } /** An ANSI-related test case. */ protected case class AnsiTestCase( - name: String, inputFile: String, resultFile: String) extends TestCase with AnsiTest + name: String, inputFile: String, resultFile: String) extends TestCase with AnsiTest { + override def asAnalyzerTest(newName: String, newResultFile: String): TestCase = + AnsiAnalyzerTestCase(newName, inputFile, newResultFile) + } /** An analyzer test that shows the analyzed plan string as output. */ protected case class AnalyzerTestCase( @@ -213,32 +217,70 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper /** A PostgreSQL test case. */ protected case class PgSQLTestCase( - name: String, inputFile: String, resultFile: String) extends TestCase with PgSQLTest + name: String, inputFile: String, resultFile: String) extends TestCase with PgSQLTest { + override def asAnalyzerTest(newName: String, newResultFile: String): TestCase = + PgSQLAnalyzerTestCase(newName, inputFile, newResultFile) + } /** A UDF test case. */ protected case class UDFTestCase( name: String, inputFile: String, resultFile: String, - udf: TestUDF) extends TestCase with UDFTest + udf: TestUDF) extends TestCase with UDFTest { + override def asAnalyzerTest(newName: String, newResultFile: String): TestCase = + UDFAnalyzerTestCase(newName, inputFile, newResultFile, udf) + } /** A UDAF test case. */ protected case class UDAFTestCase( name: String, inputFile: String, resultFile: String, - udf: TestUDF) extends TestCase with UDFTest + udf: TestUDF) extends TestCase with UDFTest { + override def asAnalyzerTest(newName: String, newResultFile: String): TestCase = + UDAFAnalyzerTestCase(newName, inputFile, newResultFile, udf) + } /** A UDF PostgreSQL test case. */ protected case class UDFPgSQLTestCase( name: String, inputFile: String, resultFile: String, - udf: TestUDF) extends TestCase with UDFTest with PgSQLTest + udf: TestUDF) extends TestCase with UDFTest with PgSQLTest { + override def asAnalyzerTest(newName: String, newResultFile: String): TestCase = + UDFPgSQLAnalyzerTestCase(newName, inputFile, newResultFile, udf) + } /** An date time test case with default timestamp as TimestampNTZType */ protected case class TimestampNTZTestCase( - name: String, inputFile: String, resultFile: String) extends TestCase with TimestampNTZTest + name: String, inputFile: String, resultFile: String) extends TestCase with TimestampNTZTest { + override def asAnalyzerTest(newName: String, newResultFile: String): TestCase = + TimestampNTZAnalyzerTestCase(newName, inputFile, newResultFile) + } + + /** These are versions of the above test cases, but only exercising analysis. */ + protected case class RegularAnalyzerTestCase( + name: String, inputFile: String, resultFile: String) + extends AnalyzerTest + protected case class AnsiAnalyzerTestCase( + name: String, inputFile: String, resultFile: String) + extends AnalyzerTest with AnsiTest + protected case class PgSQLAnalyzerTestCase( + name: String, inputFile: String, resultFile: String) + extends AnalyzerTest with PgSQLTest + protected case class UDFAnalyzerTestCase( + name: String, inputFile: String, resultFile: String, udf: TestUDF) + extends AnalyzerTest with UDFTest + protected case class UDAFAnalyzerTestCase( + name: String, inputFile: String, resultFile: String, udf: TestUDF) + extends AnalyzerTest with UDFTest + protected case class UDFPgSQLAnalyzerTestCase( + name: String, inputFile: String, resultFile: String, udf: TestUDF) + extends AnalyzerTest with UDFTest with PgSQLTest + protected case class TimestampNTZAnalyzerTestCase( + name: String, inputFile: String, resultFile: String) + extends AnalyzerTest with TimestampNTZTest protected def createScalaTestCase(testCase: TestCase): Unit = { if (ignoreList.exists(t => @@ -475,7 +517,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper withClue(clue) { testCase match { - case _: AnalyzerTestCase => + case _: AnalyzerTest => readGoldenFileAndCompareResults(testCase.resultFile, outputs, AnalyzerOutput) case _ => readGoldenFileAndCompareResults(testCase.resultFile, outputs, ExecutionOutput) @@ -492,7 +534,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper val testCaseName = absPath.stripPrefix(inputFilePath).stripPrefix(File.separator) val analyzerTestCaseName = s"${testCaseName}_analyzer_test" - val newTestCase = if (file.getAbsolutePath.startsWith( + // Create test cases of test types that depend on the input filename. + val newTestCases: Seq[TestCase] = if (file.getAbsolutePath.startsWith( s"$inputFilePath${File.separator}udf${File.separator}postgreSQL")) { Seq(TestScalaUDF("udf"), TestPythonUDF("udf"), TestScalarPandasUDF("udf")).map { udf => UDFPgSQLTestCase( @@ -517,10 +560,19 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper } else { RegularTestCase(testCaseName, absPath, resultFile) :: Nil } - if (analyzerTestCaseList.contains(file.getName.toLowerCase(Locale.ROOT))) { - AnalyzerTestCase(analyzerTestCaseName, absPath, analyzerResultFile) +: newTestCase - } else { - newTestCase + // Also include a copy of each of the above test cases as an analyzer test. + newTestCases.flatMap { test => + test match { + case _: UDAFTestCase => + // Skip creating analyzer test cases for UDAF tests as they are hard to update locally. + Seq(test) + case _ => + Seq( + test, + test.asAnalyzerTest( + newName = s"${test.name}_analyzer_test", + newResultFile = analyzerResultFile)) + } } }.sortBy(_.name) }