diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index bd449930e3b33..4d23b4f2c5665 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -190,6 +190,16 @@ "The does not support ordering on type ." ] }, + "IN_SUBQUERY_DATA_TYPE_MISMATCH" : { + "message" : [ + "The data type of one or more elements in the left hand side of an IN subquery is not compatible with the data type of the output of the subquery. Mismatched columns: [], left side: [], right side: []." + ] + }, + "IN_SUBQUERY_LENGTH_MISMATCH" : { + "message" : [ + "The number of columns in the left hand side of an IN subquery does not match the number of columns in the output of subquery. Left hand side columns(length: ): [], right hand side columns(length: ): []." + ] + }, "MAP_CONCAT_DIFF_TYPES" : { "message" : [ "The should all be of type map, but it's ." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index f506acde7c260..7efac183a674f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -19,10 +19,12 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion} +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.trees.TernaryLike import org.apache.spark.sql.catalyst.trees.TreePattern.{CASE_WHEN, IF, TreePattern} +import org.apache.spark.sql.catalyst.util.TypeUtils.{toSQLExpr, toSQLId, toSQLType} import org.apache.spark.sql.types._ // scalastyle:off line.size.limit @@ -60,12 +62,24 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi override def checkInputDataTypes(): TypeCheckResult = { if (predicate.dataType != BooleanType) { - TypeCheckResult.TypeCheckFailure( - "type of predicate expression in If should be boolean, " + - s"not ${predicate.dataType.catalogString}") + DataTypeMismatch( + errorSubClass = "UNEXPECTED_INPUT_TYPE", + messageParameters = Map( + "paramIndex" -> "1", + "requiredType" -> toSQLType(BooleanType), + "inputSql" -> toSQLExpr(predicate), + "inputType" -> toSQLType(predicate.dataType) + ) + ) } else if (!TypeCoercion.haveSameType(inputTypesForMerging)) { - TypeCheckResult.TypeCheckFailure(s"differing types in '$sql' " + - s"(${trueValue.dataType.catalogString} and ${falseValue.dataType.catalogString}).") + DataTypeMismatch( + errorSubClass = "DATA_DIFF_TYPES", + messageParameters = Map( + "functionName" -> toSQLId(prettyName), + "dataType" -> Seq(trueValue.dataType, + falseValue.dataType).map(toSQLType).mkString("[", ", ", "]") + ) + ) } else { TypeCheckResult.TypeCheckSuccess } @@ -172,17 +186,24 @@ case class CaseWhen( TypeCheckResult.TypeCheckSuccess } else { val index = branches.indexWhere(_._1.dataType != BooleanType) - TypeCheckResult.TypeCheckFailure( - s"WHEN expressions in CaseWhen should all be boolean type, " + - s"but the ${index + 1}th when expression's type is ${branches(index)._1}") + DataTypeMismatch( + errorSubClass = "UNEXPECTED_INPUT_TYPE", + messageParameters = Map( + "paramIndex" -> (index + 1).toString, + "requiredType" -> toSQLType(BooleanType), + "inputSql" -> toSQLExpr(branches(index)._1), + "inputType" -> toSQLType(branches(index)._1.dataType) + ) + ) } } else { - val branchesStr = branches.map(_._2.dataType).map(dt => s"WHEN ... THEN ${dt.catalogString}") - .mkString(" ") - val elseStr = elseValue.map(expr => s" ELSE ${expr.dataType.catalogString}").getOrElse("") - TypeCheckResult.TypeCheckFailure( - "THEN and ELSE expressions should all be same type or coercible to a common type," + - s" got CASE $branchesStr$elseStr END") + DataTypeMismatch( + errorSubClass = "DATA_DIFF_TYPES", + messageParameters = Map( + "functionName" -> toSQLId(prettyName), + "dataType" -> inputTypesForMerging.map(toSQLType).mkString("[", ", ", "]") + ) + ) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 899ece6f5297d..aab3fddb4fc2a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -22,7 +22,9 @@ import scala.collection.immutable.TreeSet import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReference +import org.apache.spark.sql.catalyst.expressions.Cast._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project} @@ -31,7 +33,6 @@ import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ - /** * A base class for generated/interpreted predicate */ @@ -375,16 +376,15 @@ case class InSubquery(values: Seq[Expression], query: ListQuery) override def checkInputDataTypes(): TypeCheckResult = { if (values.length != query.childOutputs.length) { - TypeCheckResult.TypeCheckFailure( - s""" - |The number of columns in the left hand side of an IN subquery does not match the - |number of columns in the output of subquery. - |#columns in left hand side: ${values.length}. - |#columns in right hand side: ${query.childOutputs.length}. - |Left side columns: - |[${values.map(_.sql).mkString(", ")}]. - |Right side columns: - |[${query.childOutputs.map(_.sql).mkString(", ")}].""".stripMargin) + DataTypeMismatch( + errorSubClass = "IN_SUBQUERY_LENGTH_MISMATCH", + messageParameters = Map( + "leftLength" -> values.length.toString, + "rightLength" -> query.childOutputs.length.toString, + "leftColumns" -> values.map(toSQLExpr(_)).mkString(", "), + "rightColumns" -> query.childOutputs.map(toSQLExpr(_)).mkString(", ") + ) + ) } else if (!DataType.equalsStructurally( query.dataType, value.dataType, ignoreNullability = true)) { @@ -393,16 +393,14 @@ case class InSubquery(values: Seq[Expression], query: ListQuery) Seq(s"(${l.sql}:${l.dataType.catalogString}, ${r.sql}:${r.dataType.catalogString})") case _ => None } - TypeCheckResult.TypeCheckFailure( - s""" - |The data type of one or more elements in the left hand side of an IN subquery - |is not compatible with the data type of the output of the subquery - |Mismatched columns: - |[${mismatchedColumns.mkString(", ")}] - |Left side: - |[${values.map(_.dataType.catalogString).mkString(", ")}]. - |Right side: - |[${query.childOutputs.map(_.dataType.catalogString).mkString(", ")}].""".stripMargin) + DataTypeMismatch( + errorSubClass = "IN_SUBQUERY_DATA_TYPE_MISMATCH", + messageParameters = Map( + "mismatchedColumns" -> mismatchedColumns.mkString(", "), + "leftType" -> values.map(left => toSQLType(left.dataType)).mkString(", "), + "rightType" -> query.childOutputs.map(right => toSQLType(right.dataType)).mkString(", ") + ) + ) } else { TypeUtils.checkForOrderingExpr(value.dataType, prettyName) } @@ -450,8 +448,13 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { val mismatchOpt = list.find(l => !DataType.equalsStructurally(l.dataType, value.dataType, ignoreNullability = true)) if (mismatchOpt.isDefined) { - TypeCheckResult.TypeCheckFailure(s"Arguments must be same type but were: " + - s"${value.dataType.catalogString} != ${mismatchOpt.get.dataType.catalogString}") + DataTypeMismatch( + errorSubClass = "DATA_DIFF_TYPES", + messageParameters = Map( + "functionName" -> toSQLId(prettyName), + "dataType" -> children.map(child => toSQLType(child.dataType)).mkString("[", ", ", "]") + ) + ) } else { TypeUtils.checkForOrderingExpr(value.dataType, prettyName) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index c1106a2654518..c63c232eac895 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -329,7 +329,13 @@ class AnalysisSuite extends AnalysisTest with Matchers { val plan = Project(Alias(In(Literal(null), Seq(Literal(true), Literal(1))), "a")() :: Nil, LocalRelation() ) - assertAnalysisError(plan, Seq("data type mismatch: Arguments must be same type")) + assertAnalysisErrorClass( + plan, + "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + Map( + "functionName" -> "`in`", + "dataType" -> "[\"VOID\", \"BOOLEAN\", \"INT\"]", + "sqlExpr" -> "\"(NULL IN (true, 1))\"")) } test("SPARK-11725: correctly handle null inputs for ScalaUDF") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index 9bc765df75e31..83139ab719fe0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -351,23 +351,62 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer ) ) - assertError(If($"intField", $"stringField", $"stringField"), - "type of predicate expression in If should be boolean") - assertError(If($"booleanField", $"intField", $"booleanField"), - "data type mismatch") + assert(If(Literal(1), Literal("a"), Literal("b")).checkInputDataTypes() == + DataTypeMismatch( + errorSubClass = "UNEXPECTED_INPUT_TYPE", + messageParameters = Map( + "paramIndex" -> "1", + "requiredType" -> toSQLType(BooleanType), + "inputSql" -> "\"1\"", + "inputType" -> "\"INT\"" + ) + ) + ) - assertError( - CaseWhen(Seq(($"booleanField".attr, $"intField".attr), - ($"booleanField".attr, $"mapField".attr))), - "THEN and ELSE expressions should all be same type or coercible to a common type") - assertError( - CaseKeyWhen($"intField", Seq($"intField", $"stringField", - $"intField", $"mapField")), - "THEN and ELSE expressions should all be same type or coercible to a common type") - assertError( - CaseWhen(Seq(($"booleanField".attr, $"intField".attr), - ($"intField".attr, $"intField".attr))), - "WHEN expressions in CaseWhen should all be boolean type") + assert(If(Literal(true), Literal(1), Literal(false)).checkInputDataTypes() == + DataTypeMismatch( + errorSubClass = "DATA_DIFF_TYPES", + messageParameters = Map( + "functionName" -> "`if`", + "dataType" -> "[\"INT\", \"BOOLEAN\"]" + ) + ) + ) + + assert(CaseWhen(Seq((Literal(true), Literal(1)), + (Literal(true), Literal("a")))).checkInputDataTypes() == + DataTypeMismatch( + errorSubClass = "DATA_DIFF_TYPES", + messageParameters = Map( + "functionName" -> "`casewhen`", + "dataType" -> "[\"INT\", \"STRING\"]" + ) + ) + ) + + assert(CaseKeyWhen(Literal(1), Seq(Literal(1), Literal("a"), + Literal(2), Literal(3))).checkInputDataTypes() == + DataTypeMismatch( + errorSubClass = "DATA_DIFF_TYPES", + messageParameters = Map( + "functionName" -> "`casewhen`", + "dataType" -> "[\"STRING\", \"INT\"]" + ) + ) + ) + + assert(CaseWhen(Seq((Literal(true), Literal(1)), + (Literal(2), Literal(3)))).checkInputDataTypes() == + DataTypeMismatch( + errorSubClass = "UNEXPECTED_INPUT_TYPE", + messageParameters = Map( + "paramIndex" -> "2", + "requiredType" -> "\"BOOLEAN\"", + "inputSql" -> "\"2\"", + "inputType" -> "\"INT\"" + ) + ) + ) } test("check types for aggregates") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala index 71cf3d115fc54..e7f7c370dcb97 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.types._ @@ -238,16 +238,19 @@ class ConditionalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val checkResult1 = CaseWhen(Seq((Literal.FalseLiteral, caseVal1), (Literal.FalseLiteral, caseVal2))).checkInputDataTypes() - assert(checkResult1.isInstanceOf[TypeCheckResult.TypeCheckFailure]) - assert(checkResult1.asInstanceOf[TypeCheckResult.TypeCheckFailure].message - .contains("CASE WHEN ... THEN struct WHEN ... THEN struct END")) + assert(checkResult1 == DataTypeMismatch( + errorSubClass = "DATA_DIFF_TYPES", + messageParameters = Map( + "functionName" -> "`casewhen`", + "dataType" -> "[\"STRUCT\", \"STRUCT\"]"))) val checkResult2 = CaseWhen(Seq((Literal.FalseLiteral, caseVal1), (Literal.FalseLiteral, caseVal2)), Some(elseVal)).checkInputDataTypes() - assert(checkResult2.isInstanceOf[TypeCheckResult.TypeCheckFailure]) - assert(checkResult2.asInstanceOf[TypeCheckResult.TypeCheckFailure].message - .contains("CASE WHEN ... THEN struct WHEN ... THEN struct " + - "ELSE struct END")) + assert(checkResult2 == DataTypeMismatch( + errorSubClass = "DATA_DIFF_TYPES", + messageParameters = Map( + "functionName" -> "`casewhen`", + "dataType" -> "[\"STRUCT\", \"STRUCT\", \"STRUCT\"]"))) } test("SPARK-27917 test semantic equals of CaseWhen") { diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java index 7f9fdbd592ecd..1836cc403c39e 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java @@ -17,19 +17,23 @@ package test.org.apache.spark.sql; +import java.util.*; + +import com.google.common.collect.Maps; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + import org.apache.spark.api.java.function.FilterFunction; +import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.test.TestSparkSession; import org.apache.spark.sql.types.StructType; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.*; import static org.apache.spark.sql.types.DataTypes.*; @@ -79,12 +83,13 @@ public void isInCollectionCheckExceptionMessage() { createStructField("a", IntegerType, false), createStructField("b", createArrayType(IntegerType, false), false))); Dataset df = spark.createDataFrame(rows, schema); - Exception e = Assert.assertThrows(Exception.class, + AnalysisException e = Assert.assertThrows(AnalysisException.class, () -> df.filter(df.col("a").isInCollection(Arrays.asList(new Column("b"))))); - Arrays.asList("cannot resolve", - "due to data type mismatch: Arguments must be same type but were") - .forEach(s -> - Assert.assertTrue(e.getMessage().toLowerCase(Locale.ROOT) - .contains(s.toLowerCase(Locale.ROOT)))); + Assert.assertTrue(e.getErrorClass().equals("DATATYPE_MISMATCH.DATA_DIFF_TYPES")); + Map messageParameters = new HashMap(); + messageParameters.put("functionName", "`in`"); + messageParameters.put("dataType", "[\"INT\", \"ARRAY\"]"); + messageParameters.put("sqlExpr", "\"(a IN (b))\""); + Assert.assertTrue(Maps.difference(e.getMessageParameters(), messageParameters).areEqual()); } } diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-basic.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-basic.sql.out index 6fc9f786f4126..3f6798c5d42bf 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-basic.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-basic.sql.out @@ -39,11 +39,13 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_LENGTH_MISMATCH", "messageParameters" : { - "hint" : "", - "msg" : "\nThe number of columns in the left hand side of an IN subquery does not match the\nnumber of columns in the output of subquery.\n#columns in left hand side: 2.\n#columns in right hand side: 1.\nLeft side columns:\n[tab_a.a1, tab_a.b1].\nRight side columns:\n[`named_struct(a2, a2, b2, b2)`].", - "sqlExpr" : "(named_struct('a1', tab_a.a1, 'b1', tab_a.b1) IN (listquery()))" + "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" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out index 0a58ac7734c01..77900ed7dc33d 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out @@ -114,11 +114,13 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_LENGTH_MISMATCH", "messageParameters" : { - "hint" : "", - "msg" : "\nThe number of columns in the left hand side of an IN subquery does not match the\nnumber of columns in the output of subquery.\n#columns in left hand side: 1.\n#columns in right hand side: 2.\nLeft side columns:\n[t1.t1a].\nRight side columns:\n[t2.t2a, t2.t2b].", - "sqlExpr" : "(t1.t1a IN (listquery(t1.t1a)))" + "leftColumns" : "\"t1a\"", + "leftLength" : "1", + "rightColumns" : "\"t2a\", \"t2b\"", + "rightLength" : "2", + "sqlExpr" : "\"(t1a IN (listquery(t1a)))\"" }, "queryContext" : [ { "objectType" : "", @@ -141,11 +143,13 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_LENGTH_MISMATCH", "messageParameters" : { - "hint" : "", - "msg" : "\nThe number of columns in the left hand side of an IN subquery does not match the\nnumber of columns in the output of subquery.\n#columns in left hand side: 2.\n#columns in right hand side: 1.\nLeft side columns:\n[t1.t1a, t1.t1b].\nRight side columns:\n[t2.t2a].", - "sqlExpr" : "(named_struct('t1a', t1.t1a, 't1b', t1.t1b) IN (listquery(t1.t1a)))" + "leftColumns" : "\"t1a\", \"t1b\"", + "leftLength" : "2", + "rightColumns" : "\"t2a\"", + "rightLength" : "1", + "sqlExpr" : "\"(named_struct('t1a', t1a, 't1b', t1b) IN (listquery(t1a)))\"" }, "queryContext" : [ { "objectType" : "", @@ -169,11 +173,12 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_DATA_TYPE_MISMATCH", "messageParameters" : { - "hint" : "", - "msg" : "\nThe data type of one or more elements in the left hand side of an IN subquery\nis not compatible with the data type of the output of the subquery\nMismatched columns:\n[(t4.t4a:double, t5.t5a:timestamp), (t4.t4c:string, t5.t5c:bigint)]\nLeft side:\n[double, string, string].\nRight side:\n[timestamp, string, bigint].", - "sqlExpr" : "(named_struct('t4a', t4.t4a, 't4b', t4.t4b, 't4c', t4.t4c) IN (listquery()))" + "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" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out index 344e5af545ca0..e8743f40eed58 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out @@ -78,11 +78,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN tinyint ELSE binary END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST('2' AS BINARY) END" + "dataType" : "[\"TINYINT\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS BINARY) END\"" }, "queryContext" : [ { "objectType" : "", @@ -101,11 +101,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN tinyint ELSE boolean END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS BOOLEAN) END" + "dataType" : "[\"TINYINT\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS BOOLEAN) END\"" }, "queryContext" : [ { "objectType" : "", @@ -124,11 +124,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN tinyint ELSE timestamp END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END" + "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" : "", @@ -147,11 +147,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN tinyint ELSE date END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END" + "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" : "", @@ -234,11 +234,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN smallint ELSE binary END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST('2' AS BINARY) END" + "dataType" : "[\"SMALLINT\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS BINARY) END\"" }, "queryContext" : [ { "objectType" : "", @@ -257,11 +257,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN smallint ELSE boolean END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS BOOLEAN) END" + "dataType" : "[\"SMALLINT\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS BOOLEAN) END\"" }, "queryContext" : [ { "objectType" : "", @@ -280,11 +280,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN smallint ELSE timestamp END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END" + "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" : "", @@ -303,11 +303,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN smallint ELSE date END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END" + "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" : "", @@ -390,11 +390,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN int ELSE binary END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS INT) ELSE CAST('2' AS BINARY) END" + "dataType" : "[\"INT\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS BINARY) END\"" }, "queryContext" : [ { "objectType" : "", @@ -413,11 +413,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN int ELSE boolean END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS BOOLEAN) END" + "dataType" : "[\"INT\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS BOOLEAN) END\"" }, "queryContext" : [ { "objectType" : "", @@ -436,11 +436,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN int ELSE timestamp END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS INT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END" + "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" : "", @@ -459,11 +459,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN int ELSE date END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS INT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END" + "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" : "", @@ -546,11 +546,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN bigint ELSE binary END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST('2' AS BINARY) END" + "dataType" : "[\"BIGINT\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS BINARY) END\"" }, "queryContext" : [ { "objectType" : "", @@ -569,11 +569,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN bigint ELSE boolean END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS BOOLEAN) END" + "dataType" : "[\"BIGINT\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS BOOLEAN) END\"" }, "queryContext" : [ { "objectType" : "", @@ -592,11 +592,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN bigint ELSE timestamp END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END" + "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" : "", @@ -615,11 +615,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN bigint ELSE date END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END" + "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" : "", @@ -702,11 +702,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN float ELSE binary END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST('2' AS BINARY) END" + "dataType" : "[\"FLOAT\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS BINARY) END\"" }, "queryContext" : [ { "objectType" : "", @@ -725,11 +725,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN float ELSE boolean END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS BOOLEAN) END" + "dataType" : "[\"FLOAT\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS BOOLEAN) END\"" }, "queryContext" : [ { "objectType" : "", @@ -748,11 +748,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN float ELSE timestamp END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END" + "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" : "", @@ -771,11 +771,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN float ELSE date END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END" + "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" : "", @@ -858,11 +858,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN double ELSE binary END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST('2' AS BINARY) END" + "dataType" : "[\"DOUBLE\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS BINARY) END\"" }, "queryContext" : [ { "objectType" : "", @@ -881,11 +881,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN double ELSE boolean END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS BOOLEAN) END" + "dataType" : "[\"DOUBLE\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS BOOLEAN) END\"" }, "queryContext" : [ { "objectType" : "", @@ -904,11 +904,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN double ELSE timestamp END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END" + "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" : "", @@ -927,11 +927,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN double ELSE date END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST('2017-12-11 09:30:00' AS DATE) END" + "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" : "", @@ -1014,11 +1014,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN decimal(10,0) ELSE binary END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST('2' AS BINARY) END" + "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" : "", @@ -1037,11 +1037,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN decimal(10,0) ELSE boolean END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS BOOLEAN) END" + "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" : "", @@ -1060,11 +1060,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN decimal(10,0) ELSE timestamp END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END" + "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" : "", @@ -1083,11 +1083,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN decimal(10,0) ELSE date END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST('2017-12-11 09:30:00' AS DATE) END" + "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" : "", @@ -1170,11 +1170,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN string ELSE binary END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST('2' AS BINARY) END" + "dataType" : "[\"STRING\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS BINARY) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1193,11 +1193,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN string ELSE boolean END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS BOOLEAN) END" + "dataType" : "[\"STRING\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS BOOLEAN) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1232,11 +1232,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE tinyint END", - "sqlExpr" : "CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS TINYINT) END" + "dataType" : "[\"BINARY\", \"TINYINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS TINYINT) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1255,11 +1255,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE smallint END", - "sqlExpr" : "CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS SMALLINT) END" + "dataType" : "[\"BINARY\", \"SMALLINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS SMALLINT) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1278,11 +1278,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE int END", - "sqlExpr" : "CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS INT) END" + "dataType" : "[\"BINARY\", \"INT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS INT) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1301,11 +1301,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE bigint END", - "sqlExpr" : "CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS BIGINT) END" + "dataType" : "[\"BINARY\", \"BIGINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS BIGINT) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1324,11 +1324,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE float END", - "sqlExpr" : "CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS FLOAT) END" + "dataType" : "[\"BINARY\", \"FLOAT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS FLOAT) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1347,11 +1347,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE double END", - "sqlExpr" : "CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS DOUBLE) END" + "dataType" : "[\"BINARY\", \"DOUBLE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS DOUBLE) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1370,11 +1370,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE decimal(10,0) END", - "sqlExpr" : "CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS DECIMAL(10,0)) END" + "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" : "", @@ -1393,11 +1393,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE string END", - "sqlExpr" : "CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS STRING) END" + "dataType" : "[\"BINARY\", \"STRING\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS STRING) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1424,11 +1424,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE boolean END", - "sqlExpr" : "CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS BOOLEAN) END" + "dataType" : "[\"BINARY\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS BOOLEAN) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1447,11 +1447,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE timestamp END", - "sqlExpr" : "CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END" + "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" : "", @@ -1470,11 +1470,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE date END", - "sqlExpr" : "CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST('2017-12-11 09:30:00' AS DATE) END" + "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" : "", @@ -1493,11 +1493,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE tinyint END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS TINYINT) END" + "dataType" : "[\"BOOLEAN\", \"TINYINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS TINYINT) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1516,11 +1516,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE smallint END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS SMALLINT) END" + "dataType" : "[\"BOOLEAN\", \"SMALLINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS SMALLINT) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1539,11 +1539,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE int END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS INT) END" + "dataType" : "[\"BOOLEAN\", \"INT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS INT) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1562,11 +1562,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE bigint END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS BIGINT) END" + "dataType" : "[\"BOOLEAN\", \"BIGINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS BIGINT) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1585,11 +1585,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE float END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS FLOAT) END" + "dataType" : "[\"BOOLEAN\", \"FLOAT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS FLOAT) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1608,11 +1608,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE double END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS DOUBLE) END" + "dataType" : "[\"BOOLEAN\", \"DOUBLE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS DOUBLE) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1631,11 +1631,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE decimal(10,0) END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS DECIMAL(10,0)) END" + "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" : "", @@ -1654,11 +1654,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE string END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS STRING) END" + "dataType" : "[\"BOOLEAN\", \"STRING\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS STRING) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1677,11 +1677,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE binary END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST('2' AS BINARY) END" + "dataType" : "[\"BOOLEAN\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS BINARY) END\"" }, "queryContext" : [ { "objectType" : "", @@ -1708,11 +1708,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE timestamp END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END" + "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" : "", @@ -1731,11 +1731,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE date END", - "sqlExpr" : "CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST('2017-12-11 09:30:00' AS DATE) END" + "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" : "", @@ -1754,11 +1754,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE tinyint END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS TINYINT) END" + "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" : "", @@ -1777,11 +1777,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE smallint END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS SMALLINT) END" + "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" : "", @@ -1800,11 +1800,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE int END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS INT) END" + "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" : "", @@ -1823,11 +1823,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE bigint END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS BIGINT) END" + "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" : "", @@ -1846,11 +1846,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE float END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS FLOAT) END" + "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" : "", @@ -1869,11 +1869,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE double END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS DOUBLE) END" + "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" : "", @@ -1892,11 +1892,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE decimal(10,0) END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS DECIMAL(10,0)) END" + "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" : "", @@ -1923,11 +1923,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE binary END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST('2' AS BINARY) END" + "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" : "", @@ -1946,11 +1946,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE boolean END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS BOOLEAN) END" + "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" : "", @@ -1985,11 +1985,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE tinyint END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS TINYINT) END" + "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" : "", @@ -2008,11 +2008,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE smallint END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS SMALLINT) END" + "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" : "", @@ -2031,11 +2031,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE int END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS INT) END" + "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" : "", @@ -2054,11 +2054,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE bigint END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS BIGINT) END" + "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" : "", @@ -2077,11 +2077,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE float END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS FLOAT) END" + "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" : "", @@ -2100,11 +2100,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE double END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS DOUBLE) END" + "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" : "", @@ -2123,11 +2123,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE decimal(10,0) END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS DECIMAL(10,0)) END" + "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" : "", @@ -2154,11 +2154,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE binary END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST('2' AS BINARY) END" + "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" : "", @@ -2177,11 +2177,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE boolean END", - "sqlExpr" : "CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS BOOLEAN) END" + "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" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out index cb968205895ce..a85a0de29e7f4 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out @@ -78,11 +78,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS TINYINT), CAST('2' AS BINARY)))' (tinyint and binary).", - "sqlExpr" : "(IF(true, CAST(1 AS TINYINT), CAST('2' AS BINARY)))" + "dataType" : "[\"TINYINT\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS TINYINT), CAST(2 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -101,11 +101,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS TINYINT), CAST(2 AS BOOLEAN)))' (tinyint and boolean).", - "sqlExpr" : "(IF(true, CAST(1 AS TINYINT), CAST(2 AS BOOLEAN)))" + "dataType" : "[\"TINYINT\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS TINYINT), CAST(2 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -124,11 +124,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS TINYINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (tinyint and timestamp).", - "sqlExpr" : "(IF(true, CAST(1 AS TINYINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"TINYINT\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS TINYINT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -147,11 +147,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS TINYINT), CAST('2017-12-11 09:30:00' AS DATE)))' (tinyint and date).", - "sqlExpr" : "(IF(true, CAST(1 AS TINYINT), CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"TINYINT\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS TINYINT), CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -234,11 +234,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS SMALLINT), CAST('2' AS BINARY)))' (smallint and binary).", - "sqlExpr" : "(IF(true, CAST(1 AS SMALLINT), CAST('2' AS BINARY)))" + "dataType" : "[\"SMALLINT\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS SMALLINT), CAST(2 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -257,11 +257,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS SMALLINT), CAST(2 AS BOOLEAN)))' (smallint and boolean).", - "sqlExpr" : "(IF(true, CAST(1 AS SMALLINT), CAST(2 AS BOOLEAN)))" + "dataType" : "[\"SMALLINT\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS SMALLINT), CAST(2 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -280,11 +280,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS SMALLINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (smallint and timestamp).", - "sqlExpr" : "(IF(true, CAST(1 AS SMALLINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"SMALLINT\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS SMALLINT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -303,11 +303,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS SMALLINT), CAST('2017-12-11 09:30:00' AS DATE)))' (smallint and date).", - "sqlExpr" : "(IF(true, CAST(1 AS SMALLINT), CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"SMALLINT\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS SMALLINT), CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -390,11 +390,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS INT), CAST('2' AS BINARY)))' (int and binary).", - "sqlExpr" : "(IF(true, CAST(1 AS INT), CAST('2' AS BINARY)))" + "dataType" : "[\"INT\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS INT), CAST(2 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -413,11 +413,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS INT), CAST(2 AS BOOLEAN)))' (int and boolean).", - "sqlExpr" : "(IF(true, CAST(1 AS INT), CAST(2 AS BOOLEAN)))" + "dataType" : "[\"INT\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS INT), CAST(2 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -436,11 +436,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS INT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (int and timestamp).", - "sqlExpr" : "(IF(true, CAST(1 AS INT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"INT\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS INT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -459,11 +459,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS INT), CAST('2017-12-11 09:30:00' AS DATE)))' (int and date).", - "sqlExpr" : "(IF(true, CAST(1 AS INT), CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"INT\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS INT), CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -546,11 +546,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS BIGINT), CAST('2' AS BINARY)))' (bigint and binary).", - "sqlExpr" : "(IF(true, CAST(1 AS BIGINT), CAST('2' AS BINARY)))" + "dataType" : "[\"BIGINT\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BIGINT), CAST(2 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -569,11 +569,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS BIGINT), CAST(2 AS BOOLEAN)))' (bigint and boolean).", - "sqlExpr" : "(IF(true, CAST(1 AS BIGINT), CAST(2 AS BOOLEAN)))" + "dataType" : "[\"BIGINT\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BIGINT), CAST(2 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -592,11 +592,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS BIGINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (bigint and timestamp).", - "sqlExpr" : "(IF(true, CAST(1 AS BIGINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"BIGINT\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BIGINT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -615,11 +615,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS BIGINT), CAST('2017-12-11 09:30:00' AS DATE)))' (bigint and date).", - "sqlExpr" : "(IF(true, CAST(1 AS BIGINT), CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"BIGINT\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BIGINT), CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -702,11 +702,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS FLOAT), CAST('2' AS BINARY)))' (float and binary).", - "sqlExpr" : "(IF(true, CAST(1 AS FLOAT), CAST('2' AS BINARY)))" + "dataType" : "[\"FLOAT\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS FLOAT), CAST(2 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -725,11 +725,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS FLOAT), CAST(2 AS BOOLEAN)))' (float and boolean).", - "sqlExpr" : "(IF(true, CAST(1 AS FLOAT), CAST(2 AS BOOLEAN)))" + "dataType" : "[\"FLOAT\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS FLOAT), CAST(2 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -748,11 +748,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS FLOAT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (float and timestamp).", - "sqlExpr" : "(IF(true, CAST(1 AS FLOAT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"FLOAT\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS FLOAT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -771,11 +771,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS FLOAT), CAST('2017-12-11 09:30:00' AS DATE)))' (float and date).", - "sqlExpr" : "(IF(true, CAST(1 AS FLOAT), CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"FLOAT\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS FLOAT), CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -858,11 +858,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS DOUBLE), CAST('2' AS BINARY)))' (double and binary).", - "sqlExpr" : "(IF(true, CAST(1 AS DOUBLE), CAST('2' AS BINARY)))" + "dataType" : "[\"DOUBLE\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DOUBLE), CAST(2 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -881,11 +881,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS DOUBLE), CAST(2 AS BOOLEAN)))' (double and boolean).", - "sqlExpr" : "(IF(true, CAST(1 AS DOUBLE), CAST(2 AS BOOLEAN)))" + "dataType" : "[\"DOUBLE\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DOUBLE), CAST(2 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -904,11 +904,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS DOUBLE), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (double and timestamp).", - "sqlExpr" : "(IF(true, CAST(1 AS DOUBLE), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"DOUBLE\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DOUBLE), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -927,11 +927,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS DOUBLE), CAST('2017-12-11 09:30:00' AS DATE)))' (double and date).", - "sqlExpr" : "(IF(true, CAST(1 AS DOUBLE), CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"DOUBLE\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DOUBLE), CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1014,11 +1014,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS DECIMAL(10,0)), CAST('2' AS BINARY)))' (decimal(10,0) and binary).", - "sqlExpr" : "(IF(true, CAST(1 AS DECIMAL(10,0)), CAST('2' AS BINARY)))" + "dataType" : "[\"DECIMAL(10,0)\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1037,11 +1037,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS BOOLEAN)))' (decimal(10,0) and boolean).", - "sqlExpr" : "(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS BOOLEAN)))" + "dataType" : "[\"DECIMAL(10,0)\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1060,11 +1060,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (decimal(10,0) and timestamp).", - "sqlExpr" : "(IF(true, CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "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" : "", @@ -1083,11 +1083,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00' AS DATE)))' (decimal(10,0) and date).", - "sqlExpr" : "(IF(true, CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00' AS DATE)))" + "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" : "", @@ -1170,11 +1170,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS STRING), CAST('2' AS BINARY)))' (string and binary).", - "sqlExpr" : "(IF(true, CAST(1 AS STRING), CAST('2' AS BINARY)))" + "dataType" : "[\"STRING\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS STRING), CAST(2 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1193,11 +1193,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS STRING), CAST(2 AS BOOLEAN)))' (string and boolean).", - "sqlExpr" : "(IF(true, CAST(1 AS STRING), CAST(2 AS BOOLEAN)))" + "dataType" : "[\"STRING\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS STRING), CAST(2 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1232,11 +1232,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS TINYINT)))' (binary and tinyint).", - "sqlExpr" : "(IF(true, CAST('1' AS BINARY), CAST(2 AS TINYINT)))" + "dataType" : "[\"BINARY\", \"TINYINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS TINYINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1255,11 +1255,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS SMALLINT)))' (binary and smallint).", - "sqlExpr" : "(IF(true, CAST('1' AS BINARY), CAST(2 AS SMALLINT)))" + "dataType" : "[\"BINARY\", \"SMALLINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS SMALLINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1278,11 +1278,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS INT)))' (binary and int).", - "sqlExpr" : "(IF(true, CAST('1' AS BINARY), CAST(2 AS INT)))" + "dataType" : "[\"BINARY\", \"INT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS INT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1301,11 +1301,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS BIGINT)))' (binary and bigint).", - "sqlExpr" : "(IF(true, CAST('1' AS BINARY), CAST(2 AS BIGINT)))" + "dataType" : "[\"BINARY\", \"BIGINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS BIGINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1324,11 +1324,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS FLOAT)))' (binary and float).", - "sqlExpr" : "(IF(true, CAST('1' AS BINARY), CAST(2 AS FLOAT)))" + "dataType" : "[\"BINARY\", \"FLOAT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS FLOAT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1347,11 +1347,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS DOUBLE)))' (binary and double).", - "sqlExpr" : "(IF(true, CAST('1' AS BINARY), CAST(2 AS DOUBLE)))" + "dataType" : "[\"BINARY\", \"DOUBLE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS DOUBLE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1370,11 +1370,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS DECIMAL(10,0))))' (binary and decimal(10,0)).", - "sqlExpr" : "(IF(true, CAST('1' AS BINARY), CAST(2 AS DECIMAL(10,0))))" + "dataType" : "[\"BINARY\", \"DECIMAL(10,0)\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS DECIMAL(10,0))))\"" }, "queryContext" : [ { "objectType" : "", @@ -1393,11 +1393,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS STRING)))' (binary and string).", - "sqlExpr" : "(IF(true, CAST('1' AS BINARY), CAST(2 AS STRING)))" + "dataType" : "[\"BINARY\", \"STRING\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS STRING)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1424,11 +1424,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS BOOLEAN)))' (binary and boolean).", - "sqlExpr" : "(IF(true, CAST('1' AS BINARY), CAST(2 AS BOOLEAN)))" + "dataType" : "[\"BINARY\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1447,11 +1447,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('1' AS BINARY), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (binary and timestamp).", - "sqlExpr" : "(IF(true, CAST('1' AS BINARY), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"BINARY\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1470,11 +1470,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('1' AS BINARY), CAST('2017-12-11 09:30:00' AS DATE)))' (binary and date).", - "sqlExpr" : "(IF(true, CAST('1' AS BINARY), CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"BINARY\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1493,11 +1493,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS TINYINT)))' (boolean and tinyint).", - "sqlExpr" : "(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS TINYINT)))" + "dataType" : "[\"BOOLEAN\", \"TINYINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS TINYINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1516,11 +1516,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS SMALLINT)))' (boolean and smallint).", - "sqlExpr" : "(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS SMALLINT)))" + "dataType" : "[\"BOOLEAN\", \"SMALLINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS SMALLINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1539,11 +1539,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS INT)))' (boolean and int).", - "sqlExpr" : "(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS INT)))" + "dataType" : "[\"BOOLEAN\", \"INT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS INT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1562,11 +1562,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS BIGINT)))' (boolean and bigint).", - "sqlExpr" : "(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS BIGINT)))" + "dataType" : "[\"BOOLEAN\", \"BIGINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS BIGINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1585,11 +1585,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS FLOAT)))' (boolean and float).", - "sqlExpr" : "(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS FLOAT)))" + "dataType" : "[\"BOOLEAN\", \"FLOAT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS FLOAT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1608,11 +1608,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS DOUBLE)))' (boolean and double).", - "sqlExpr" : "(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS DOUBLE)))" + "dataType" : "[\"BOOLEAN\", \"DOUBLE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS DOUBLE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1631,11 +1631,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS DECIMAL(10,0))))' (boolean and decimal(10,0)).", - "sqlExpr" : "(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS DECIMAL(10,0))))" + "dataType" : "[\"BOOLEAN\", \"DECIMAL(10,0)\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS DECIMAL(10,0))))\"" }, "queryContext" : [ { "objectType" : "", @@ -1654,11 +1654,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS STRING)))' (boolean and string).", - "sqlExpr" : "(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS STRING)))" + "dataType" : "[\"BOOLEAN\", \"STRING\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS STRING)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1677,11 +1677,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST('2' AS BINARY)))' (boolean and binary).", - "sqlExpr" : "(IF(true, CAST(1 AS BOOLEAN), CAST('2' AS BINARY)))" + "dataType" : "[\"BOOLEAN\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1708,11 +1708,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (boolean and timestamp).", - "sqlExpr" : "(IF(true, CAST(1 AS BOOLEAN), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"BOOLEAN\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1731,11 +1731,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST('2017-12-11 09:30:00' AS DATE)))' (boolean and date).", - "sqlExpr" : "(IF(true, CAST(1 AS BOOLEAN), CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"BOOLEAN\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1754,11 +1754,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS TINYINT)))' (timestamp and tinyint).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS TINYINT)))" + "dataType" : "[\"TIMESTAMP\", \"TINYINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS TINYINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1777,11 +1777,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS SMALLINT)))' (timestamp and smallint).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS SMALLINT)))" + "dataType" : "[\"TIMESTAMP\", \"SMALLINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS SMALLINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1800,11 +1800,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS INT)))' (timestamp and int).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS INT)))" + "dataType" : "[\"TIMESTAMP\", \"INT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS INT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1823,11 +1823,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS BIGINT)))' (timestamp and bigint).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS BIGINT)))" + "dataType" : "[\"TIMESTAMP\", \"BIGINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS BIGINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1846,11 +1846,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS FLOAT)))' (timestamp and float).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS FLOAT)))" + "dataType" : "[\"TIMESTAMP\", \"FLOAT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS FLOAT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1869,11 +1869,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS DOUBLE)))' (timestamp and double).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS DOUBLE)))" + "dataType" : "[\"TIMESTAMP\", \"DOUBLE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS DOUBLE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1892,11 +1892,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS DECIMAL(10,0))))' (timestamp and decimal(10,0)).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS DECIMAL(10,0))))" + "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" : "", @@ -1923,11 +1923,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST('2' AS BINARY)))' (timestamp and binary).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST('2' AS BINARY)))" + "dataType" : "[\"TIMESTAMP\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1946,11 +1946,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS BOOLEAN)))' (timestamp and boolean).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS BOOLEAN)))" + "dataType" : "[\"TIMESTAMP\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1985,11 +1985,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS TINYINT)))' (date and tinyint).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS TINYINT)))" + "dataType" : "[\"DATE\", \"TINYINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS TINYINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2008,11 +2008,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS SMALLINT)))' (date and smallint).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS SMALLINT)))" + "dataType" : "[\"DATE\", \"SMALLINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS SMALLINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2031,11 +2031,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS INT)))' (date and int).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS INT)))" + "dataType" : "[\"DATE\", \"INT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS INT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2054,11 +2054,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS BIGINT)))' (date and bigint).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS BIGINT)))" + "dataType" : "[\"DATE\", \"BIGINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS BIGINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2077,11 +2077,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS FLOAT)))' (date and float).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS FLOAT)))" + "dataType" : "[\"DATE\", \"FLOAT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS FLOAT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2100,11 +2100,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS DOUBLE)))' (date and double).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS DOUBLE)))" + "dataType" : "[\"DATE\", \"DOUBLE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS DOUBLE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2123,11 +2123,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS DECIMAL(10,0))))' (date and decimal(10,0)).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS DECIMAL(10,0))))" + "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" : "", @@ -2154,11 +2154,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST('2' AS BINARY)))' (date and binary).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST('2' AS BINARY)))" + "dataType" : "[\"DATE\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2177,11 +2177,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS BOOLEAN)))' (date and boolean).", - "sqlExpr" : "(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS BOOLEAN)))" + "dataType" : "[\"DATE\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out index c5fb9f0622598..2540dc42c6d61 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out @@ -78,11 +78,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: tinyint != binary", - "sqlExpr" : "(CAST(1 AS TINYINT) IN (CAST('1' AS BINARY)))" + "dataType" : "[\"TINYINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -101,11 +101,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: tinyint != boolean", - "sqlExpr" : "(CAST(1 AS TINYINT) IN (CAST(1 AS BOOLEAN)))" + "dataType" : "[\"TINYINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -124,11 +124,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: tinyint != timestamp", - "sqlExpr" : "(CAST(1 AS TINYINT) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"TINYINT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -147,11 +147,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: tinyint != date", - "sqlExpr" : "(CAST(1 AS TINYINT) IN (CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"TINYINT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -234,11 +234,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: smallint != binary", - "sqlExpr" : "(CAST(1 AS SMALLINT) IN (CAST('1' AS BINARY)))" + "dataType" : "[\"SMALLINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -257,11 +257,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: smallint != boolean", - "sqlExpr" : "(CAST(1 AS SMALLINT) IN (CAST(1 AS BOOLEAN)))" + "dataType" : "[\"SMALLINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -280,11 +280,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: smallint != timestamp", - "sqlExpr" : "(CAST(1 AS SMALLINT) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"SMALLINT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -303,11 +303,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: smallint != date", - "sqlExpr" : "(CAST(1 AS SMALLINT) IN (CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"SMALLINT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -390,11 +390,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: int != binary", - "sqlExpr" : "(CAST(1 AS INT) IN (CAST('1' AS BINARY)))" + "dataType" : "[\"INT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -413,11 +413,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: int != boolean", - "sqlExpr" : "(CAST(1 AS INT) IN (CAST(1 AS BOOLEAN)))" + "dataType" : "[\"INT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -436,11 +436,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: int != timestamp", - "sqlExpr" : "(CAST(1 AS INT) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"INT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -459,11 +459,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: int != date", - "sqlExpr" : "(CAST(1 AS INT) IN (CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"INT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -546,11 +546,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: bigint != binary", - "sqlExpr" : "(CAST(1 AS BIGINT) IN (CAST('1' AS BINARY)))" + "dataType" : "[\"BIGINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -569,11 +569,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: bigint != boolean", - "sqlExpr" : "(CAST(1 AS BIGINT) IN (CAST(1 AS BOOLEAN)))" + "dataType" : "[\"BIGINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -592,11 +592,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: bigint != timestamp", - "sqlExpr" : "(CAST(1 AS BIGINT) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"BIGINT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -615,11 +615,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: bigint != date", - "sqlExpr" : "(CAST(1 AS BIGINT) IN (CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"BIGINT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -702,11 +702,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: float != binary", - "sqlExpr" : "(CAST(1 AS FLOAT) IN (CAST('1' AS BINARY)))" + "dataType" : "[\"FLOAT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -725,11 +725,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: float != boolean", - "sqlExpr" : "(CAST(1 AS FLOAT) IN (CAST(1 AS BOOLEAN)))" + "dataType" : "[\"FLOAT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -748,11 +748,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: float != timestamp", - "sqlExpr" : "(CAST(1 AS FLOAT) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"FLOAT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -771,11 +771,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: float != date", - "sqlExpr" : "(CAST(1 AS FLOAT) IN (CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"FLOAT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -858,11 +858,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: double != binary", - "sqlExpr" : "(CAST(1 AS DOUBLE) IN (CAST('1' AS BINARY)))" + "dataType" : "[\"DOUBLE\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -881,11 +881,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: double != boolean", - "sqlExpr" : "(CAST(1 AS DOUBLE) IN (CAST(1 AS BOOLEAN)))" + "dataType" : "[\"DOUBLE\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -904,11 +904,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: double != timestamp", - "sqlExpr" : "(CAST(1 AS DOUBLE) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"DOUBLE\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -927,11 +927,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: double != date", - "sqlExpr" : "(CAST(1 AS DOUBLE) IN (CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"DOUBLE\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1014,11 +1014,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: decimal(10,0) != binary", - "sqlExpr" : "(CAST(1 AS DECIMAL(10,0)) IN (CAST('1' AS BINARY)))" + "dataType" : "[\"DECIMAL(10,0)\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1037,11 +1037,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: decimal(10,0) != boolean", - "sqlExpr" : "(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS BOOLEAN)))" + "dataType" : "[\"DECIMAL(10,0)\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1060,11 +1060,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: decimal(10,0) != timestamp", - "sqlExpr" : "(CAST(1 AS DECIMAL(10,0)) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "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" : "", @@ -1083,11 +1083,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: decimal(10,0) != date", - "sqlExpr" : "(CAST(1 AS DECIMAL(10,0)) IN (CAST('2017-12-11 09:30:00' AS DATE)))" + "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" : "", @@ -1170,11 +1170,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: string != binary", - "sqlExpr" : "(CAST(1 AS STRING) IN (CAST('1' AS BINARY)))" + "dataType" : "[\"STRING\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS STRING) IN (CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1193,11 +1193,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: string != boolean", - "sqlExpr" : "(CAST(1 AS STRING) IN (CAST(1 AS BOOLEAN)))" + "dataType" : "[\"STRING\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS STRING) IN (CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1232,11 +1232,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != tinyint", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST(1 AS TINYINT)))" + "dataType" : "[\"BINARY\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS TINYINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1255,11 +1255,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != smallint", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST(1 AS SMALLINT)))" + "dataType" : "[\"BINARY\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS SMALLINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1278,11 +1278,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != int", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST(1 AS INT)))" + "dataType" : "[\"BINARY\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS INT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1301,11 +1301,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != bigint", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST(1 AS BIGINT)))" + "dataType" : "[\"BINARY\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BIGINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1324,11 +1324,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != float", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST(1 AS FLOAT)))" + "dataType" : "[\"BINARY\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS FLOAT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1347,11 +1347,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != double", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST(1 AS DOUBLE)))" + "dataType" : "[\"BINARY\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS DOUBLE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1370,11 +1370,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != decimal(10,0)", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST(1 AS DECIMAL(10,0))))" + "dataType" : "[\"BINARY\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS DECIMAL(10,0))))\"" }, "queryContext" : [ { "objectType" : "", @@ -1393,11 +1393,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != string", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST(1 AS STRING)))" + "dataType" : "[\"BINARY\", \"STRING\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS STRING)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1424,11 +1424,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != boolean", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST(1 AS BOOLEAN)))" + "dataType" : "[\"BINARY\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1447,11 +1447,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != timestamp", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"BINARY\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1470,11 +1470,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != date", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"BINARY\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1493,11 +1493,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != tinyint", - "sqlExpr" : "(true IN (CAST(1 AS TINYINT)))" + "dataType" : "[\"BOOLEAN\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS TINYINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1516,11 +1516,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != smallint", - "sqlExpr" : "(true IN (CAST(1 AS SMALLINT)))" + "dataType" : "[\"BOOLEAN\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS SMALLINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1539,11 +1539,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != int", - "sqlExpr" : "(true IN (CAST(1 AS INT)))" + "dataType" : "[\"BOOLEAN\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS INT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1562,11 +1562,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != bigint", - "sqlExpr" : "(true IN (CAST(1 AS BIGINT)))" + "dataType" : "[\"BOOLEAN\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS BIGINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1585,11 +1585,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != float", - "sqlExpr" : "(true IN (CAST(1 AS FLOAT)))" + "dataType" : "[\"BOOLEAN\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS FLOAT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1608,11 +1608,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != double", - "sqlExpr" : "(true IN (CAST(1 AS DOUBLE)))" + "dataType" : "[\"BOOLEAN\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS DOUBLE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1631,11 +1631,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != decimal(10,0)", - "sqlExpr" : "(true IN (CAST(1 AS DECIMAL(10,0))))" + "dataType" : "[\"BOOLEAN\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS DECIMAL(10,0))))\"" }, "queryContext" : [ { "objectType" : "", @@ -1654,11 +1654,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != string", - "sqlExpr" : "(true IN (CAST(1 AS STRING)))" + "dataType" : "[\"BOOLEAN\", \"STRING\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS STRING)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1677,11 +1677,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != binary", - "sqlExpr" : "(true IN (CAST('1' AS BINARY)))" + "dataType" : "[\"BOOLEAN\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1708,11 +1708,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != timestamp", - "sqlExpr" : "(true IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "dataType" : "[\"BOOLEAN\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1731,11 +1731,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != date", - "sqlExpr" : "(true IN (CAST('2017-12-11 09:30:00' AS DATE)))" + "dataType" : "[\"BOOLEAN\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1754,11 +1754,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != tinyint", - "sqlExpr" : "(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS TINYINT)))" + "dataType" : "[\"TIMESTAMP\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS TINYINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1777,11 +1777,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != smallint", - "sqlExpr" : "(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS SMALLINT)))" + "dataType" : "[\"TIMESTAMP\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS SMALLINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1800,11 +1800,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != int", - "sqlExpr" : "(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS INT)))" + "dataType" : "[\"TIMESTAMP\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS INT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1823,11 +1823,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != bigint", - "sqlExpr" : "(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS BIGINT)))" + "dataType" : "[\"TIMESTAMP\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS BIGINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1846,11 +1846,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != float", - "sqlExpr" : "(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS FLOAT)))" + "dataType" : "[\"TIMESTAMP\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS FLOAT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1869,11 +1869,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != double", - "sqlExpr" : "(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS DOUBLE)))" + "dataType" : "[\"TIMESTAMP\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS DOUBLE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1892,11 +1892,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != decimal(10,0)", - "sqlExpr" : "(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS DECIMAL(10,0))))" + "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" : "", @@ -1923,11 +1923,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != binary", - "sqlExpr" : "(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST('2' AS BINARY)))" + "dataType" : "[\"TIMESTAMP\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1946,11 +1946,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != boolean", - "sqlExpr" : "(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS BOOLEAN)))" + "dataType" : "[\"TIMESTAMP\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -1985,11 +1985,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != tinyint", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS TINYINT)))" + "dataType" : "[\"DATE\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS TINYINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2008,11 +2008,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != smallint", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS SMALLINT)))" + "dataType" : "[\"DATE\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS SMALLINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2031,11 +2031,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != int", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS INT)))" + "dataType" : "[\"DATE\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS INT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2054,11 +2054,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != bigint", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS BIGINT)))" + "dataType" : "[\"DATE\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS BIGINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2077,11 +2077,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != float", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS FLOAT)))" + "dataType" : "[\"DATE\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS FLOAT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2100,11 +2100,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != double", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS DOUBLE)))" + "dataType" : "[\"DATE\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS DOUBLE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2123,11 +2123,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != decimal(10,0)", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS DECIMAL(10,0))))" + "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" : "", @@ -2154,11 +2154,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != binary", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2' AS BINARY)))" + "dataType" : "[\"DATE\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2177,11 +2177,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != boolean", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS BOOLEAN)))" + "dataType" : "[\"DATE\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2280,11 +2280,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: tinyint != binary", - "sqlExpr" : "(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST('1' AS BINARY)))" + "dataType" : "[\"TINYINT\", \"TINYINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2303,11 +2303,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: tinyint != boolean", - "sqlExpr" : "(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS BOOLEAN)))" + "dataType" : "[\"TINYINT\", \"TINYINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2326,11 +2326,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: tinyint != timestamp", - "sqlExpr" : "(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "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" : "", @@ -2349,11 +2349,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: tinyint != date", - "sqlExpr" : "(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST('2017-12-11 09:30:00' AS DATE)))" + "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" : "", @@ -2436,11 +2436,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: smallint != binary", - "sqlExpr" : "(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST('1' AS BINARY)))" + "dataType" : "[\"SMALLINT\", \"SMALLINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2459,11 +2459,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: smallint != boolean", - "sqlExpr" : "(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS BOOLEAN)))" + "dataType" : "[\"SMALLINT\", \"SMALLINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2482,11 +2482,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: smallint != timestamp", - "sqlExpr" : "(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "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" : "", @@ -2505,11 +2505,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: smallint != date", - "sqlExpr" : "(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST('2017-12-11 09:30:00' AS DATE)))" + "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" : "", @@ -2592,11 +2592,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: int != binary", - "sqlExpr" : "(CAST(1 AS INT) IN (CAST(1 AS INT), CAST('1' AS BINARY)))" + "dataType" : "[\"INT\", \"INT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2615,11 +2615,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: int != boolean", - "sqlExpr" : "(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS BOOLEAN)))" + "dataType" : "[\"INT\", \"INT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2638,11 +2638,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: int != timestamp", - "sqlExpr" : "(CAST(1 AS INT) IN (CAST(1 AS INT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "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" : "", @@ -2661,11 +2661,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: int != date", - "sqlExpr" : "(CAST(1 AS INT) IN (CAST(1 AS INT), CAST('2017-12-11 09:30:00' AS DATE)))" + "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" : "", @@ -2748,11 +2748,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: bigint != binary", - "sqlExpr" : "(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST('1' AS BINARY)))" + "dataType" : "[\"BIGINT\", \"BIGINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2771,11 +2771,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: bigint != boolean", - "sqlExpr" : "(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS BOOLEAN)))" + "dataType" : "[\"BIGINT\", \"BIGINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2794,11 +2794,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: bigint != timestamp", - "sqlExpr" : "(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "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" : "", @@ -2817,11 +2817,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: bigint != date", - "sqlExpr" : "(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST('2017-12-11 09:30:00' AS DATE)))" + "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" : "", @@ -2904,11 +2904,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: float != binary", - "sqlExpr" : "(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST('1' AS BINARY)))" + "dataType" : "[\"FLOAT\", \"FLOAT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2927,11 +2927,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: float != boolean", - "sqlExpr" : "(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS BOOLEAN)))" + "dataType" : "[\"FLOAT\", \"FLOAT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -2950,11 +2950,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: float != timestamp", - "sqlExpr" : "(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "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" : "", @@ -2973,11 +2973,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: float != date", - "sqlExpr" : "(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST('2017-12-11 09:30:00' AS DATE)))" + "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" : "", @@ -3060,11 +3060,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: double != binary", - "sqlExpr" : "(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST('1' AS BINARY)))" + "dataType" : "[\"DOUBLE\", \"DOUBLE\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3083,11 +3083,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: double != boolean", - "sqlExpr" : "(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS BOOLEAN)))" + "dataType" : "[\"DOUBLE\", \"DOUBLE\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3106,11 +3106,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: double != timestamp", - "sqlExpr" : "(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "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" : "", @@ -3129,11 +3129,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: double != date", - "sqlExpr" : "(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST('2017-12-11 09:30:00' AS DATE)))" + "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" : "", @@ -3216,11 +3216,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: decimal(10,0) != binary", - "sqlExpr" : "(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST('1' AS BINARY)))" + "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" : "", @@ -3239,11 +3239,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: decimal(10,0) != boolean", - "sqlExpr" : "(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS BOOLEAN)))" + "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" : "", @@ -3262,11 +3262,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: decimal(10,0) != timestamp", - "sqlExpr" : "(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "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" : "", @@ -3285,11 +3285,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: decimal(10,0) != date", - "sqlExpr" : "(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00' AS DATE)))" + "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" : "", @@ -3372,11 +3372,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: string != binary", - "sqlExpr" : "(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST('1' AS BINARY)))" + "dataType" : "[\"STRING\", \"STRING\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3395,11 +3395,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: string != boolean", - "sqlExpr" : "(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BOOLEAN)))" + "dataType" : "[\"STRING\", \"STRING\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3434,11 +3434,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != tinyint", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS TINYINT)))" + "dataType" : "[\"BINARY\", \"BINARY\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS TINYINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3457,11 +3457,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != smallint", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS SMALLINT)))" + "dataType" : "[\"BINARY\", \"BINARY\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS SMALLINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3480,11 +3480,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != int", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS INT)))" + "dataType" : "[\"BINARY\", \"BINARY\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS INT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3503,11 +3503,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != bigint", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS BIGINT)))" + "dataType" : "[\"BINARY\", \"BINARY\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS BIGINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3526,11 +3526,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != float", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS FLOAT)))" + "dataType" : "[\"BINARY\", \"BINARY\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS FLOAT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3549,11 +3549,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != double", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS DOUBLE)))" + "dataType" : "[\"BINARY\", \"BINARY\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS DOUBLE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3572,11 +3572,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != decimal(10,0)", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS DECIMAL(10,0))))" + "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" : "", @@ -3595,11 +3595,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != string", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS STRING)))" + "dataType" : "[\"BINARY\", \"BINARY\", \"STRING\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS STRING)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3626,11 +3626,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != boolean", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS BOOLEAN)))" + "dataType" : "[\"BINARY\", \"BINARY\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS BOOLEAN)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3649,11 +3649,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != timestamp", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "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" : "", @@ -3672,11 +3672,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: binary != date", - "sqlExpr" : "(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST('2017-12-11 09:30:00' AS DATE)))" + "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" : "", @@ -3695,11 +3695,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != tinyint", - "sqlExpr" : "(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS TINYINT)))" + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS TINYINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3718,11 +3718,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != smallint", - "sqlExpr" : "(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS SMALLINT)))" + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS SMALLINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3741,11 +3741,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != int", - "sqlExpr" : "(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS INT)))" + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS INT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3764,11 +3764,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != bigint", - "sqlExpr" : "(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS BIGINT)))" + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS BIGINT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3787,11 +3787,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != float", - "sqlExpr" : "(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS FLOAT)))" + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS FLOAT)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3810,11 +3810,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != double", - "sqlExpr" : "(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS DOUBLE)))" + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS DOUBLE)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3833,11 +3833,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != decimal(10,0)", - "sqlExpr" : "(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS DECIMAL(10,0))))" + "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" : "", @@ -3856,11 +3856,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != string", - "sqlExpr" : "(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS STRING)))" + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"STRING\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS STRING)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3879,11 +3879,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != binary", - "sqlExpr" : "(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST('1' AS BINARY)))" + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS BINARY)))\"" }, "queryContext" : [ { "objectType" : "", @@ -3910,11 +3910,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != timestamp", - "sqlExpr" : "(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))" + "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" : "", @@ -3933,11 +3933,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: boolean != date", - "sqlExpr" : "(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST('2017-12-11 09:30:00' AS DATE)))" + "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" : "", @@ -3956,11 +3956,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != tinyint", - "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)))" + "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" : "", @@ -3979,11 +3979,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != smallint", - "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)))" + "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" : "", @@ -4002,11 +4002,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != int", - "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)))" + "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" : "", @@ -4025,11 +4025,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != bigint", - "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)))" + "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" : "", @@ -4048,11 +4048,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != float", - "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)))" + "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" : "", @@ -4071,11 +4071,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != double", - "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)))" + "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" : "", @@ -4094,11 +4094,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != decimal(10,0)", - "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))))" + "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" : "", @@ -4125,11 +4125,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != binary", - "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)))" + "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" : "", @@ -4148,11 +4148,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: timestamp != boolean", - "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)))" + "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" : "", @@ -4187,11 +4187,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != tinyint", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST(1 AS TINYINT)))" + "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" : "", @@ -4210,11 +4210,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != smallint", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST(1 AS SMALLINT)))" + "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" : "", @@ -4233,11 +4233,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != int", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST(1 AS INT)))" + "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" : "", @@ -4256,11 +4256,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != bigint", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST(1 AS BIGINT)))" + "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" : "", @@ -4279,11 +4279,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != float", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST(1 AS FLOAT)))" + "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" : "", @@ -4302,11 +4302,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != double", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST(1 AS DOUBLE)))" + "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" : "", @@ -4325,11 +4325,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != decimal(10,0)", - "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))))" + "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" : "", @@ -4356,11 +4356,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != binary", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST('1' AS BINARY)))" + "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" : "", @@ -4379,11 +4379,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "messageParameters" : { - "hint" : "", - "msg" : "Arguments must be same type but were: date != boolean", - "sqlExpr" : "(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST(1 AS BOOLEAN)))" + "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" : "", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index f109b7ff90481..d7ebb9003884f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -450,14 +450,16 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { df.collect().toSeq.filter(r => r.getInt(0) == 3 || r.getInt(0) == 1)) val df2 = Seq((1, Seq(1)), (2, Seq(2)), (3, Seq(3))).toDF("a", "b") - - val e = intercept[AnalysisException] { - df2.filter($"a".isin($"b")) - } - Seq("cannot resolve", "due to data type mismatch: Arguments must be same type but were") - .foreach { s => - assert(e.getMessage.toLowerCase(Locale.ROOT).contains(s.toLowerCase(Locale.ROOT))) - } + checkError( + exception = intercept[AnalysisException] { + df2.filter($"a".isin($"b")) + }, + errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + parameters = Map( + "functionName" -> "`in`", + "dataType" -> "[\"INT\", \"ARRAY\"]", + "sqlExpr" -> "\"(a IN (b))\"") + ) } test("IN/INSET with bytes, shorts, ints, dates") { @@ -515,14 +517,16 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { df.collect().toSeq.filter(r => r.getInt(0) == 3 || r.getInt(0) == 1)) val df2 = Seq((1, Seq(1)), (2, Seq(2)), (3, Seq(3))).toDF("a", "b") - - val e = intercept[AnalysisException] { - df2.filter($"a".isInCollection(Seq($"b"))) - } - Seq("cannot resolve", "due to data type mismatch: Arguments must be same type but were") - .foreach { s => - assert(e.getMessage.toLowerCase(Locale.ROOT).contains(s.toLowerCase(Locale.ROOT))) - } + checkError( + exception = intercept[AnalysisException] { + df2.filter($"a".isInCollection(Seq($"b"))) + }, + errorClass = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + parameters = Map( + "functionName" -> "`in`", + "dataType" -> "[\"INT\", \"ARRAY\"]", + "sqlExpr" -> "\"(a IN (b))\"") + ) } } }