From bf5103ae5bb938102850160048e2a1656e648244 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Sat, 10 Sep 2022 18:03:07 +0300 Subject: [PATCH] [SPARK-40324][SQL] Provide query context in AnalysisException ### What changes were proposed in this pull request? This is a major PR for extending AnalysisException with query context. After changes, we can: * migrate analysis errors to error classes * change the actual error messages of AnalysisException, showing the query context from original query instead of an unresolved logical plan. ### Why are the changes needed? Make it possible for better error messages in analysis exceptions. ### Does this PR introduce _any_ user-facing change? No. This is an extension of the framework. We will have more improvements based on this one. ### How was this patch tested? Existing UT Closes #37841 from gengliangwang/analysisContext. Authored-by: Gengliang Wang Signed-off-by: Max Gekk --- .../org/apache/spark/SparkFunSuite.scala | 10 + project/MimaExcludes.scala | 6 +- .../apache/spark/sql/AnalysisException.scala | 18 +- .../sql/catalyst/analysis/Analyzer.scala | 3 +- .../spark/sql/catalyst/analysis/package.scala | 2 +- .../spark/sql/catalyst/trees/TreeNode.scala | 7 + .../sql-tests/results/ansi/interval.sql.out | 252 +- .../sql-tests/results/ansi/timestamp.sql.out | 36 +- .../results/columnresolution-negative.sql.out | 36 +- .../sql-tests/results/datetime-legacy.sql.out | 36 +- .../sql-tests/results/group-by-filter.sql.out | 18 +- .../results/group-by-ordinal.sql.out | 81 +- .../sql-tests/results/group-by.sql.out | 45 +- .../sql-tests/results/grouping_set.sql.out | 9 +- .../sql-tests/results/having.sql.out | 9 +- .../sql-tests/results/interval.sql.out | 252 +- .../sql-tests/results/join-lateral.sql.out | 36 +- .../sql-tests/results/natural-join.sql.out | 9 +- .../resources/sql-tests/results/pivot.sql.out | 18 +- .../postgreSQL/aggregates_part1.sql.out | 9 +- .../results/postgreSQL/create_view.sql.out | 9 +- .../sql-tests/results/postgreSQL/join.sql.out | 72 +- .../results/postgreSQL/select_having.sql.out | 9 +- .../postgreSQL/select_implicit.sql.out | 27 +- .../sql-tests/results/postgreSQL/text.sql.out | 9 +- .../results/postgreSQL/union.sql.out | 9 +- .../results/query_regex_column.sql.out | 72 +- .../invalid-correlation.sql.out | 9 +- .../sql-tests/results/table-aliases.sql.out | 9 +- .../sql-tests/results/timestamp.sql.out | 36 +- .../timestampNTZ/timestamp-ansi.sql.out | 36 +- .../results/timestampNTZ/timestamp.sql.out | 36 +- .../native/booleanEquality.sql.out | 216 +- .../native/decimalPrecision.sql.out | 2952 +++++++++++++++-- .../typeCoercion/native/division.sql.out | 720 +++- .../native/promoteStrings.sql.out | 378 ++- .../postgreSQL/udf-aggregates_part1.sql.out | 9 +- .../results/udf/postgreSQL/udf-join.sql.out | 72 +- .../udf/postgreSQL/udf-select_having.sql.out | 9 +- .../postgreSQL/udf-select_implicit.sql.out | 27 +- .../results/udf/udf-group-by.sql.out | 36 +- .../sql-tests/results/udf/udf-pivot.sql.out | 18 +- .../sql/DataFrameWindowFunctionsSuite.scala | 2 + .../apache/spark/sql/SQLInsertTestSuite.scala | 11 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 15 +- .../org/apache/spark/sql/SubquerySuite.scala | 16 +- .../errors/QueryCompilationErrorsSuite.scala | 60 +- .../sql/errors/QueryErrorsSuiteBase.scala | 3 +- .../QueryExecutionAnsiErrorsSuite.scala | 6 +- .../errors/QueryExecutionErrorsSuite.scala | 2 + .../sql/errors/QueryParsingErrorsSuite.scala | 3 +- .../spark/sql/execution/SQLViewSuite.scala | 27 +- .../command/v2/DescribeTableSuite.scala | 19 +- .../spark/sql/sources/InsertSuite.scala | 11 +- .../spark/sql/hive/HiveParquetSuite.scala | 33 +- 55 files changed, 5178 insertions(+), 692 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index c15ebab511b3b..95d13b17ee418 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -370,6 +370,16 @@ abstract class SparkFunSuite context: QueryContext): Unit = checkError(exception, errorClass, Some(errorSubClass), None, Map.empty, false, Array(context)) + protected def checkError( + exception: SparkThrowable, + errorClass: String, + errorSubClass: String, + sqlState: Option[String], + parameters: Map[String, String], + context: QueryContext): Unit = + checkError(exception, errorClass, Some(errorSubClass), sqlState, parameters, + false, Array(context)) + protected def checkError( exception: SparkThrowable, errorClass: String, diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 939fa9a9f45d2..e77f65fb7d6d3 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -85,7 +85,11 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.types.Decimal.fromStringANSI$default$3"), // [SPARK-36511][MINOR][SQL] Remove ColumnIOUtil - ProblemFilters.exclude[MissingClassProblem]("org.apache.parquet.io.ColumnIOUtil") + ProblemFilters.exclude[MissingClassProblem]("org.apache.parquet.io.ColumnIOUtil"), + + // [SPARK-40324][SQL] Provide query context in AnalysisException + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.AnalysisException.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.AnalysisException.withPosition") ) // Defulat exclude rules diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index 6c81cf8566c20..bb2ed251f3efd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -92,7 +92,8 @@ class AnalysisException protected[sql] ( startPosition = origin.startPosition, errorClass = Some(errorClass), errorSubClass = None, - messageParameters = messageParameters) + messageParameters = messageParameters, + context = origin.getQueryContext) def this( errorClass: String, @@ -115,7 +116,8 @@ class AnalysisException protected[sql] ( startPosition = origin.startPosition, errorClass = Some(errorClass), errorSubClass = Option(errorSubClass), - messageParameters = messageParameters) + messageParameters = messageParameters, + context = origin.getQueryContext) def copy( message: String = this.message, @@ -124,12 +126,16 @@ class AnalysisException protected[sql] ( plan: Option[LogicalPlan] = this.plan, cause: Option[Throwable] = this.cause, errorClass: Option[String] = this.errorClass, - messageParameters: Array[String] = this.messageParameters): AnalysisException = + messageParameters: Array[String] = this.messageParameters, + context: Array[QueryContext] = Array.empty): AnalysisException = new AnalysisException(message, line, startPosition, plan, cause, errorClass, errorSubClass, - messageParameters) + messageParameters, context) - def withPosition(line: Option[Int], startPosition: Option[Int]): AnalysisException = { - val newException = this.copy(line = line, startPosition = startPosition) + def withPosition(origin: Origin): AnalysisException = { + val newException = this.copy( + line = origin.line, + startPosition = origin.startPosition, + context = origin.getQueryContext) newException.setStackTrace(getStackTrace) newException } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 6fc9d756c998d..b4d927adecb67 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -209,7 +209,8 @@ class Analyzer(override val catalogManager: CatalogManager) analyzed } catch { case e: AnalysisException => - val ae = e.copy(plan = Option(analyzed)) + val ae = e.copy(plan = Option(analyzed), + context = analyzed.origin.getQueryContext) ae.setStackTrace(e.getStackTrace) throw ae } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index 10a7926242cbf..be6dcfac269a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -70,7 +70,7 @@ package object analysis { def withPosition[A](t: TreeNode[_])(f: => A): A = { try f catch { case a: AnalysisException => - throw a.withPosition(t.origin.line, t.origin.startPosition) + throw a.withPosition(t.origin) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 8f5858d2f4dc6..a7573fc1bd9c4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -28,6 +28,7 @@ import org.json4s.JsonAST._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ +import org.apache.spark.QueryContext import org.apache.spark.sql.catalyst.{AliasIdentifier, CatalystIdentifier} import org.apache.spark.sql.catalyst.ScalaReflection._ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, FunctionResource} @@ -68,6 +69,12 @@ case class Origin( lazy val context: SQLQueryContext = SQLQueryContext( line, startPosition, startIndex, stopIndex, sqlText, objectType, objectName) + + def getQueryContext: Array[QueryContext] = if (context.isValid) { + Array(context) + } else { + Array.empty + } } /** diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index bdab9174f8e29..9cc649878044f 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -292,7 +292,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(2 / INTERVAL '02' SECOND)\"", "left" : "\"STRING\"", "right" : "\"INTERVAL SECOND\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "select '2' / interval 2 second" + } ] } @@ -309,7 +316,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"", "left" : "\"STRING\"", "right" : "\"INTERVAL YEAR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 28, + "fragment" : "select '2' / interval 2 year" + } ] } @@ -412,7 +426,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"", "left" : "\"INT\"", "right" : "\"INTERVAL YEAR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 28, + "fragment" : "select 2 / interval '2' year" + } ] } @@ -429,7 +450,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(2 / INTERVAL '02' HOUR)\"", "left" : "\"INT\"", "right" : "\"INTERVAL HOUR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 28, + "fragment" : "select 2 / interval '2' hour" + } ] } @@ -446,7 +474,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(NULL / INTERVAL '2' YEAR)\"", "left" : "\"VOID\"", "right" : "\"INTERVAL YEAR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 31, + "fragment" : "select null / interval '2' year" + } ] } @@ -463,7 +498,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(NULL / INTERVAL '02' HOUR)\"", "left" : "\"VOID\"", "right" : "\"INTERVAL HOUR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 31, + "fragment" : "select null / interval '2' hour" + } ] } @@ -1634,7 +1676,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3 year to month)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 388, + "fragment" : "select\n interval '2' year + '3-3 year to month',\n interval '2' year - '3 month',\n '3-2 year to month' + interval '2-2' year to month,\n '3 year' - interval '2-2' year to month,\n interval '99 11:22:33.123456789' day to second + '12:12 hour to second',\n interval '99 11:22:33.123456789' day to second - '12 hour',\n '4 day' + interval '10' day,\n '4 22 day to hour' - interval '10' day" + } ] } @@ -1667,7 +1716,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 32, + "fragment" : "select interval '2' year + '3-3'" + } ] } @@ -1684,7 +1740,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '2' YEAR - 4)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "select interval '2' year - '4'" + } ] } @@ -1759,7 +1822,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '2' YEAR + str)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "select interval '2' year + str from interval_view" + } ] } @@ -1776,7 +1846,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '2' YEAR - str)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "select interval '2' year - str from interval_view" + } ] } @@ -1870,7 +1947,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)\"", "left" : "\"INTERVAL DAY\"", "right" : "\"INTERVAL YEAR TO MONTH\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "select interval '3' day - interval '2-2' year to month" + } ] } @@ -1896,7 +1980,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 + INTERVAL '2' MONTH)\"", "left" : "\"INT\"", "right" : "\"INTERVAL MONTH\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "select 1 + interval '2' month" + } ] } @@ -1922,7 +2013,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '2' MONTH - 1)\"", "left" : "\"INTERVAL MONTH\"", "right" : "\"INT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "select interval '2' month - 1" + } ] } @@ -2786,7 +2884,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' MONTH > INTERVAL '20' DAY)\"", "left" : "\"INTERVAL MONTH\"", "right" : "\"INTERVAL DAY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 42, + "fragment" : "SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS" + } ] } @@ -2803,7 +2908,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' DAY < 1)\"", "left" : "\"INTERVAL DAY\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "SELECT INTERVAL '1' DAY < '1'" + } ] } @@ -2820,7 +2932,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' DAY = 1)\"", "left" : "\"INTERVAL DAY\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "SELECT INTERVAL '1' DAY = '1'" + } ] } @@ -2837,7 +2956,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' DAY > 1)\"", "left" : "\"INTERVAL DAY\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "SELECT INTERVAL '1' DAY > '1'" + } ] } @@ -2854,7 +2980,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 < INTERVAL '1' DAY)\"", "left" : "\"STRING\"", "right" : "\"INTERVAL DAY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "SELECT '1' < INTERVAL '1' DAY" + } ] } @@ -2871,7 +3004,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 = INTERVAL '1' DAY)\"", "left" : "\"STRING\"", "right" : "\"INTERVAL DAY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "SELECT '1' = INTERVAL '1' DAY" + } ] } @@ -2888,7 +3028,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 > INTERVAL '1' DAY)\"", "left" : "\"STRING\"", "right" : "\"INTERVAL DAY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "SELECT '1' > INTERVAL '1' DAY" + } ] } @@ -2905,7 +3052,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' YEAR < 1)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "SELECT INTERVAL '1' YEAR < '1'" + } ] } @@ -2922,7 +3076,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' YEAR = 1)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "SELECT INTERVAL '1' YEAR = '1'" + } ] } @@ -2939,7 +3100,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' YEAR > 1)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "SELECT INTERVAL '1' YEAR > '1'" + } ] } @@ -2956,7 +3124,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 < INTERVAL '1' YEAR)\"", "left" : "\"STRING\"", "right" : "\"INTERVAL YEAR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "SELECT '1' < INTERVAL '1' YEAR" + } ] } @@ -2973,7 +3148,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 = INTERVAL '1' YEAR)\"", "left" : "\"STRING\"", "right" : "\"INTERVAL YEAR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "SELECT '1' = INTERVAL '1' YEAR" + } ] } @@ -2990,7 +3172,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 > INTERVAL '1' YEAR)\"", "left" : "\"STRING\"", "right" : "\"INTERVAL YEAR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "SELECT '1' > INTERVAL '1' YEAR" + } ] } @@ -3105,7 +3294,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' MONTH div INTERVAL '-1' DAY)\"", "left" : "\"INTERVAL MONTH\"", "right" : "\"INTERVAL DAY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "SELECT div(INTERVAL '1' MONTH, INTERVAL '-1' DAY)" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/timestamp.sql.out index 801d1a38d4a0d..7969ccf1162d2 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/timestamp.sql.out @@ -726,7 +726,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"", "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "actualDataType" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "select timestamp'2011-11-11 11:11:11' + '1'" + } ] } @@ -743,7 +750,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"", "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "actualDataType" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "select '1' + timestamp'2011-11-11 11:11:11'" + } ] } @@ -760,7 +774,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"", "left" : "\"TIMESTAMP\"", "right" : "\"VOID\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 44, + "fragment" : "select timestamp'2011-11-11 11:11:11' + null" + } ] } @@ -777,7 +798,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"", "left" : "\"VOID\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 44, + "fragment" : "select null + timestamp'2011-11-11 11:11:11'" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out b/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out index 3afc3f3547699..3b25d183d5d65 100644 --- a/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out @@ -165,7 +165,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`db1`.`t1`.`i1`", "proposal" : "`spark_catalog`.`mydb2`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 34, + "fragment" : "SELECT db1.t1.i1 FROM t1, mydb2.t1" + } ] } @@ -198,7 +205,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`mydb1`.`t1`", "proposal" : "`spark_catalog`.`mydb1`.`t1`.`i1`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 23, + "fragment" : "SELECT mydb1.t1 FROM t1" + } ] } @@ -224,7 +238,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`t1`", "proposal" : "`spark_catalog`.`mydb1`.`t1`.`i1`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 23, + "fragment" : "SELECT t1 FROM mydb1.t1" + } ] } @@ -249,7 +270,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`mydb1`.`t1`.`i1`", "proposal" : "`spark_catalog`.`mydb2`.`t1`.`i1`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 26, + "fragment" : "SELECT mydb1.t1.i1 FROM t1" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index 34dcea2831a62..decb0e6ac632c 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -1500,7 +1500,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"", "left" : "\"TIMESTAMP\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "select timestamp'2011-11-11 11:11:11' + '1'" + } ] } @@ -1517,7 +1524,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"", "left" : "\"DOUBLE\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "select '1' + timestamp'2011-11-11 11:11:11'" + } ] } @@ -1534,7 +1548,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"", "left" : "\"TIMESTAMP\"", "right" : "\"VOID\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 44, + "fragment" : "select timestamp'2011-11-11 11:11:11' + null" + } ] } @@ -1551,7 +1572,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"", "left" : "\"VOID\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 44, + "fragment" : "select null + timestamp'2011-11-11 11:11:11'" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out index b458ea6a2bf70..f7b1a90ad4e10 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out @@ -233,7 +233,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"a\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 56, + "stopIndex" : 65, + "fragment" : "GROUP BY b" + } ] } @@ -719,7 +726,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"a\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 65, + "stopIndex" : 78, + "fragment" : "GROUP BY a + 1" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out index 10b244c1c4395..58bdf32e02eea 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out @@ -98,7 +98,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "index" : "-1", "size" : "2" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 33, + "fragment" : "-1" + } ] } @@ -114,7 +121,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "index" : "0", "size" : "2" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 32, + "fragment" : "0" + } ] } @@ -130,7 +144,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "index" : "3", "size" : "2" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 32, + "fragment" : "3" + } ] } @@ -146,7 +167,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "index" : "3", "aggExpr" : "sum(data.b) AS `sum(b)`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 40, + "stopIndex" : 40, + "fragment" : "3" + } ] } @@ -162,7 +190,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "index" : "3", "aggExpr" : "(sum(data.b) + CAST(2 AS BIGINT)) AS `(sum(b) + 2)`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 44, + "fragment" : "3" + } ] } @@ -390,7 +425,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "index" : "-1", "size" : "3" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 46, + "fragment" : "-1" + } ] } @@ -406,7 +448,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "index" : "3", "aggExpr" : "count(1) AS `count(1)`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 45, + "fragment" : "3" + } ] } @@ -422,7 +471,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "index" : "-1", "size" : "3" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 47, + "stopIndex" : 48, + "fragment" : "-1" + } ] } @@ -438,7 +494,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "index" : "3", "aggExpr" : "count(1) AS `count(1)`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 50, + "stopIndex" : 50, + "fragment" : "3" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 4e97bd3945407..ad4106ff48997 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -48,7 +48,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"a\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 43, + "fragment" : "GROUP BY b" + } ] } @@ -118,7 +125,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"a\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 38, + "stopIndex" : 51, + "fragment" : "GROUP BY a + 1" + } ] } @@ -183,7 +197,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`non_existing`", "proposal" : "`testdata`.`a`, `testdata`.`b`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 50, + "stopIndex" : 59, + "fragment" : "GROUP BY k" + } ] } @@ -216,7 +237,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"k\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 68, + "fragment" : "GROUP BY a" + } ] } @@ -241,7 +269,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`k`", "proposal" : "`testdata`.`a`, `testdata`.`b`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 48, + "fragment" : "GROUP BY k" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out index 83b721373dae6..c024386da0c70 100644 --- a/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out @@ -171,7 +171,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"c1\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 48, + "stopIndex" : 74, + "fragment" : "GROUP BY GROUPING SETS (())" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/having.sql.out b/sql/core/src/test/resources/sql-tests/results/having.sql.out index 88cafacffdf18..4f59e111069af 100644 --- a/sql/core/src/test/resources/sql-tests/results/having.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/having.sql.out @@ -42,7 +42,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(v = array(1))\"", "left" : "\"INT\"", "right" : "\"ARRAY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 55, + "fragment" : "SELECT count(k) FROM hav GROUP BY v HAVING v = array(1)" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 06a0a14e6d458..29516de73149e 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -190,7 +190,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(2 / INTERVAL '02' SECOND)\"", "left" : "\"DOUBLE\"", "right" : "\"INTERVAL SECOND\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "select '2' / interval 2 second" + } ] } @@ -207,7 +214,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"", "left" : "\"DOUBLE\"", "right" : "\"INTERVAL YEAR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 28, + "fragment" : "select '2' / interval 2 year" + } ] } @@ -310,7 +324,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"", "left" : "\"INT\"", "right" : "\"INTERVAL YEAR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 28, + "fragment" : "select 2 / interval '2' year" + } ] } @@ -327,7 +348,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(2 / INTERVAL '02' HOUR)\"", "left" : "\"INT\"", "right" : "\"INTERVAL HOUR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 28, + "fragment" : "select 2 / interval '2' hour" + } ] } @@ -344,7 +372,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(NULL / INTERVAL '2' YEAR)\"", "left" : "\"VOID\"", "right" : "\"INTERVAL YEAR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 31, + "fragment" : "select null / interval '2' year" + } ] } @@ -361,7 +396,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(NULL / INTERVAL '02' HOUR)\"", "left" : "\"VOID\"", "right" : "\"INTERVAL HOUR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 31, + "fragment" : "select null / interval '2' hour" + } ] } @@ -1515,7 +1557,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3 year to month)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 388, + "fragment" : "select\n interval '2' year + '3-3 year to month',\n interval '2' year - '3 month',\n '3-2 year to month' + interval '2-2' year to month,\n '3 year' - interval '2-2' year to month,\n interval '99 11:22:33.123456789' day to second + '12:12 hour to second',\n interval '99 11:22:33.123456789' day to second - '12 hour',\n '4 day' + interval '10' day,\n '4 22 day to hour' - interval '10' day" + } ] } @@ -1548,7 +1597,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 32, + "fragment" : "select interval '2' year + '3-3'" + } ] } @@ -1565,7 +1621,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '2' YEAR - 4)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "select interval '2' year - '4'" + } ] } @@ -1606,7 +1669,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '2' YEAR + str)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "select interval '2' year + str from interval_view" + } ] } @@ -1623,7 +1693,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '2' YEAR - str)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "select interval '2' year - str from interval_view" + } ] } @@ -1683,7 +1760,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)\"", "left" : "\"INTERVAL DAY\"", "right" : "\"INTERVAL YEAR TO MONTH\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "select interval '3' day - interval '2-2' year to month" + } ] } @@ -1709,7 +1793,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 + INTERVAL '2' MONTH)\"", "left" : "\"INT\"", "right" : "\"INTERVAL MONTH\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "select 1 + interval '2' month" + } ] } @@ -1735,7 +1826,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '2' MONTH - 1)\"", "left" : "\"INTERVAL MONTH\"", "right" : "\"INT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "select interval '2' month - 1" + } ] } @@ -2599,7 +2697,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' MONTH > INTERVAL '20' DAY)\"", "left" : "\"INTERVAL MONTH\"", "right" : "\"INTERVAL DAY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 42, + "fragment" : "SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS" + } ] } @@ -2616,7 +2721,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' DAY < 1)\"", "left" : "\"INTERVAL DAY\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "SELECT INTERVAL '1' DAY < '1'" + } ] } @@ -2633,7 +2745,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' DAY = 1)\"", "left" : "\"INTERVAL DAY\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "SELECT INTERVAL '1' DAY = '1'" + } ] } @@ -2650,7 +2769,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' DAY > 1)\"", "left" : "\"INTERVAL DAY\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "SELECT INTERVAL '1' DAY > '1'" + } ] } @@ -2667,7 +2793,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 < INTERVAL '1' DAY)\"", "left" : "\"STRING\"", "right" : "\"INTERVAL DAY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "SELECT '1' < INTERVAL '1' DAY" + } ] } @@ -2684,7 +2817,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 = INTERVAL '1' DAY)\"", "left" : "\"STRING\"", "right" : "\"INTERVAL DAY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "SELECT '1' = INTERVAL '1' DAY" + } ] } @@ -2701,7 +2841,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 > INTERVAL '1' DAY)\"", "left" : "\"STRING\"", "right" : "\"INTERVAL DAY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "SELECT '1' > INTERVAL '1' DAY" + } ] } @@ -2718,7 +2865,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' YEAR < 1)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "SELECT INTERVAL '1' YEAR < '1'" + } ] } @@ -2735,7 +2889,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' YEAR = 1)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "SELECT INTERVAL '1' YEAR = '1'" + } ] } @@ -2752,7 +2913,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' YEAR > 1)\"", "left" : "\"INTERVAL YEAR\"", "right" : "\"STRING\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "SELECT INTERVAL '1' YEAR > '1'" + } ] } @@ -2769,7 +2937,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 < INTERVAL '1' YEAR)\"", "left" : "\"STRING\"", "right" : "\"INTERVAL YEAR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "SELECT '1' < INTERVAL '1' YEAR" + } ] } @@ -2786,7 +2961,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 = INTERVAL '1' YEAR)\"", "left" : "\"STRING\"", "right" : "\"INTERVAL YEAR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "SELECT '1' = INTERVAL '1' YEAR" + } ] } @@ -2803,7 +2985,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 > INTERVAL '1' YEAR)\"", "left" : "\"STRING\"", "right" : "\"INTERVAL YEAR\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 30, + "fragment" : "SELECT '1' > INTERVAL '1' YEAR" + } ] } @@ -2918,7 +3107,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(INTERVAL '1' MONTH div INTERVAL '-1' DAY)\"", "left" : "\"INTERVAL MONTH\"", "right" : "\"INTERVAL DAY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "SELECT div(INTERVAL '1' MONTH, INTERVAL '-1' DAY)" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out index 5180df6fcdee5..f68d18175f923 100644 --- a/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out @@ -284,7 +284,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "objectName" : "`t2`.`c1`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT * FROM t1 JOIN LATERAL (SELECT t1.c1 AS a, t2.c1 AS b) s JOIN t2 ON s.b = t2.c1" + } ] } @@ -409,7 +416,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "objectName" : "`t1`.`c1`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.c1 + t2.c1))" + } ] } @@ -425,7 +439,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "objectName" : "`c2`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 74, + "fragment" : "SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1), LATERAL (SELECT c2))" + } ] } @@ -460,7 +481,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`t1`.`c1`", "proposal" : "`spark_catalog`.`default`.`t2`.`c1`, `spark_catalog`.`default`.`t2`.`c2`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 80, + "fragment" : "SELECT * FROM t1, LATERAL (SELECT c1, (SELECT SUM(c2) FROM t2 WHERE c1 = t1.c1))" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out index debb6b626bd0b..bfcf63a318697 100644 --- a/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out @@ -236,7 +236,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`nt2`.`k`", "proposal" : "`__auto_generated_subquery_name`.`k`, `__auto_generated_subquery_name`.`v1`, `__auto_generated_subquery_name`.`v2`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT nt2.k FROM (SELECT * FROM nt1 natural join nt2)" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out index bacc541deff9e..5eb70900d465a 100644 --- a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out @@ -236,7 +236,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`year`", "proposal" : "`__auto_generated_subquery_name`.`course`, `__auto_generated_subquery_name`.`earnings`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 113, + "fragment" : "SELECT * FROM (\n SELECT course, earnings FROM courseSales\n)\nPIVOT (\n sum(earnings)\n FOR year IN (2012, 2013)\n)" + } ] } @@ -346,7 +353,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`s`", "proposal" : "`coursesales`.`year`, `coursesales`.`course`, `coursesales`.`earnings`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 75, + "fragment" : "SELECT * FROM courseSales\nPIVOT (\n sum(earnings)\n FOR year IN (s, 2013)\n)" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out index 1a6f38699844f..f4a3688348e2e 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out @@ -502,5 +502,12 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`o`.`unique1`", "proposal" : "`i`.`unique1`, `i`.`unique2`, `i`.`hundred`, `i`.`even`, `i`.`four`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 95, + "fragment" : "select\n (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)))\nfrom tenk1 o" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out index f7a98fa3aca1f..26e05a444d25f 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out @@ -75,7 +75,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "objectName" : "`FROM`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 109, + "fragment" : "CREATE VIEW key_dependent_view_no_cols AS\n SELECT FROM view_base_table GROUP BY key HAVING length(data) > 0" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out index 19e5d94f24bf1..de2a939c8a549 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out @@ -3252,7 +3252,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`y`.`f1`", "proposal" : "`j`.`f1`, `j`.`f1`, `x`.`q1`, `x`.`q2`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 81, + "fragment" : "select * from\n int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = y.f1" + } ] } @@ -3279,7 +3286,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`t1`.`uunique1`", "proposal" : "`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`hundred`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 67, + "fragment" : "select t1.uunique1 from\n tenk1 t1 join tenk2 t2 on t1.two = t2.two" + } ] } @@ -3297,7 +3311,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`t2`.`uunique1`", "proposal" : "`t2`.`unique1`, `t1`.`unique1`, `t2`.`unique2`, `t1`.`unique2`, `t2`.`hundred`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 67, + "fragment" : "select t2.uunique1 from\n tenk1 t1 join tenk2 t2 on t1.two = t2.two" + } ] } @@ -3315,7 +3336,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`uunique1`", "proposal" : "`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`even`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 64, + "fragment" : "select uunique1 from\n tenk1 t1 join tenk2 t2 on t1.two = t2.two" + } ] } @@ -3522,7 +3550,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "objectName" : "`f1`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 48, + "fragment" : "select f1,g from int4_tbl a, (select f1 as g) ss" + } ] } @@ -3538,7 +3573,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "objectName" : "`a`.`f1`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 50, + "fragment" : "select f1,g from int4_tbl a, (select a.f1 as g) ss" + } ] } @@ -3554,7 +3596,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "objectName" : "`f1`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 58, + "fragment" : "select f1,g from int4_tbl a cross join (select f1 as g) ss" + } ] } @@ -3570,7 +3619,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "objectName" : "`a`.`f1`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "select f1,g from int4_tbl a cross join (select a.f1 as g) ss" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out index 0c2b1701830cc..d717c5f7e677b 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out @@ -156,7 +156,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`a`", "proposal" : "`one`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 45, + "fragment" : "SELECT 1 AS one FROM test_having HAVING a > 1" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out index 6f1e11e0afe16..b5d1df99934c7 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out @@ -126,7 +126,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`b`", "proposal" : "`count(1)`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 53, + "stopIndex" : 62, + "fragment" : "ORDER BY b" + } ] } @@ -214,7 +221,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "index" : "3", "size" : "2" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 54, + "stopIndex" : 54, + "fragment" : "3" + } ] } @@ -346,7 +360,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`b`", "proposal" : "`count(a)`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 53, + "stopIndex" : 62, + "fragment" : "ORDER BY b" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out index 906b47d62a023..1067fb6ee18d8 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out @@ -315,7 +315,14 @@ org.apache.spark.sql.AnalysisException "parameter" : "strfmt", "functionName" : "`format_string`", "expected" : "expects %1$, %2$ and so on, but got %0$." - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "format_string('%0$s', 'Hello')" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out index 20b26d895b096..75c3fc1f53b16 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out @@ -588,7 +588,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`q2`", "proposal" : "`int8_tbl`.`q1`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 56, + "stopIndex" : 74, + "fragment" : "ORDER BY q2 LIMIT 1" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out index 16f6ad9d44931..03b8f59ab8b7c 100644 --- a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out @@ -40,7 +40,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`(a)?+.+`", "proposal" : "`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "SELECT `(a)?+.+` FROM testData2 WHERE a = 1" + } ] } @@ -57,7 +64,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`t`.`(a)?+.+`", "proposal" : "`t`.`A`, `t`.`B`, `t`.`c`, `t`.`d`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 47, + "fragment" : "SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1" + } ] } @@ -74,7 +88,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`(a|b)`", "proposal" : "`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 41, + "fragment" : "SELECT `(a|b)` FROM testData2 WHERE a = 2" + } ] } @@ -91,7 +112,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`(a|b)?+.+`", "proposal" : "`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 45, + "fragment" : "SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2" + } ] } @@ -108,7 +136,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`(a|b)?+.+`", "proposal" : "`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 38, + "fragment" : "SELECT SUM(`(a|b)?+.+`) FROM testData2" + } ] } @@ -125,7 +160,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`(a)`", "proposal" : "`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 32, + "fragment" : "SELECT SUM(`(a)`) FROM testData2" + } ] } @@ -353,7 +395,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`(a)`", "proposal" : "`testdata3`.`a`, `testdata3`.`b`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 43, + "fragment" : "GROUP BY `(a)`" + } ] } @@ -370,5 +419,12 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`(a)?+.+`", "proposal" : "`testdata3`.`a`, `testdata3`.`b`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 47, + "fragment" : "GROUP BY `(a)?+.+`" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out index 0b7581afc25aa..68dfc470edc62 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out @@ -141,5 +141,12 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`t1a`", "proposal" : "`t2`.`t2a`, `t2`.`t2b`, `t2`.`t2c`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "SELECT t1.t1a\nFROM t1\nJOIN t1_copy\nON EXISTS (SELECT 1 FROM t2 WHERE t2a > t1a)" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out index 41998dcab177e..cd54af2ae9967 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out @@ -64,7 +64,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`a`", "proposal" : "`t`.`c`, `t`.`d`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 52, + "fragment" : "SELECT a AS col1, b AS col2 FROM testData AS t(c, d)" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out index 16698dcef579d..3a8a9684c66c4 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out @@ -672,7 +672,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"", "left" : "\"TIMESTAMP\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "select timestamp'2011-11-11 11:11:11' + '1'" + } ] } @@ -689,7 +696,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"", "left" : "\"DOUBLE\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "select '1' + timestamp'2011-11-11 11:11:11'" + } ] } @@ -706,7 +720,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"", "left" : "\"TIMESTAMP\"", "right" : "\"VOID\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 44, + "fragment" : "select timestamp'2011-11-11 11:11:11' + null" + } ] } @@ -723,7 +744,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"", "left" : "\"VOID\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 44, + "fragment" : "select null + timestamp'2011-11-11 11:11:11'" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out index c16dc6c82b6a3..446687e63aea5 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out @@ -741,7 +741,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + 1)\"", "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "actualDataType" : "\"TIMESTAMP_NTZ\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "select timestamp'2011-11-11 11:11:11' + '1'" + } ] } @@ -758,7 +765,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"", "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "actualDataType" : "\"TIMESTAMP_NTZ\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "select '1' + timestamp'2011-11-11 11:11:11'" + } ] } @@ -775,7 +789,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + NULL)\"", "left" : "\"TIMESTAMP_NTZ\"", "right" : "\"VOID\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 44, + "fragment" : "select timestamp'2011-11-11 11:11:11' + null" + } ] } @@ -792,7 +813,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(NULL + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"", "left" : "\"VOID\"", "right" : "\"TIMESTAMP_NTZ\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 44, + "fragment" : "select null + timestamp'2011-11-11 11:11:11'" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp.sql.out index 54d635ba41ffc..0661df238e37d 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestampNTZ/timestamp.sql.out @@ -672,7 +672,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + 1)\"", "left" : "\"TIMESTAMP_NTZ\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "select timestamp'2011-11-11 11:11:11' + '1'" + } ] } @@ -689,7 +696,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"", "left" : "\"DOUBLE\"", "right" : "\"TIMESTAMP_NTZ\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "select '1' + timestamp'2011-11-11 11:11:11'" + } ] } @@ -706,7 +720,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + NULL)\"", "left" : "\"TIMESTAMP_NTZ\"", "right" : "\"VOID\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 44, + "fragment" : "select timestamp'2011-11-11 11:11:11' + null" + } ] } @@ -723,7 +744,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(NULL + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"", "left" : "\"VOID\"", "right" : "\"TIMESTAMP_NTZ\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 44, + "fragment" : "select null + timestamp'2011-11-11 11:11:11'" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/booleanEquality.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/booleanEquality.sql.out index ad668bd053b8c..a4a3bedf59887 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/booleanEquality.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/booleanEquality.sql.out @@ -84,7 +84,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(true = CAST(1 AS BINARY))\"", "left" : "\"BOOLEAN\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 40, + "fragment" : "SELECT true = cast('1' as binary) FROM t" + } ] } @@ -109,7 +116,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(true = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"BOOLEAN\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "SELECT true = cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -126,7 +140,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(true = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"BOOLEAN\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 56, + "fragment" : "SELECT true = cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -207,7 +228,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(true <=> CAST(1 AS BINARY))\"", "left" : "\"BOOLEAN\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 42, + "fragment" : "SELECT true <=> cast('1' as binary) FROM t" + } ] } @@ -232,7 +260,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(true <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"BOOLEAN\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 65, + "fragment" : "SELECT true <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -249,7 +284,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(true <=> CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"BOOLEAN\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 58, + "fragment" : "SELECT true <=> cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -330,7 +372,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) = true)\"", "left" : "\"BINARY\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 40, + "fragment" : "SELECT cast('1' as binary) = true FROM t" + } ] } @@ -355,7 +404,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = true)\"", "left" : "\"TIMESTAMP\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) = true FROM t" + } ] } @@ -372,7 +428,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = true)\"", "left" : "\"DATE\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 56, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) = true FROM t" + } ] } @@ -453,7 +516,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) <=> true)\"", "left" : "\"BINARY\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 42, + "fragment" : "SELECT cast('1' as binary) <=> true FROM t" + } ] } @@ -478,7 +548,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> true)\"", "left" : "\"TIMESTAMP\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 65, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> true FROM t" + } ] } @@ -495,7 +572,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> true)\"", "left" : "\"DATE\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 58, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <=> true FROM t" + } ] } @@ -576,7 +660,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(false = CAST(0 AS BINARY))\"", "left" : "\"BOOLEAN\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 41, + "fragment" : "SELECT false = cast('0' as binary) FROM t" + } ] } @@ -601,7 +692,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(false = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"BOOLEAN\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 64, + "fragment" : "SELECT false = cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -618,7 +716,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(false = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"BOOLEAN\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 57, + "fragment" : "SELECT false = cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -699,7 +804,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(false <=> CAST(0 AS BINARY))\"", "left" : "\"BOOLEAN\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "SELECT false <=> cast('0' as binary) FROM t" + } ] } @@ -724,7 +836,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(false <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"BOOLEAN\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "SELECT false <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -741,7 +860,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(false <=> CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"BOOLEAN\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 59, + "fragment" : "SELECT false <=> cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -822,7 +948,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(0 AS BINARY) = false)\"", "left" : "\"BINARY\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 41, + "fragment" : "SELECT cast('0' as binary) = false FROM t" + } ] } @@ -847,7 +980,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = false)\"", "left" : "\"TIMESTAMP\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 64, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) = false FROM t" + } ] } @@ -864,7 +1004,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = false)\"", "left" : "\"DATE\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 57, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) = false FROM t" + } ] } @@ -945,7 +1092,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(0 AS BINARY) <=> false)\"", "left" : "\"BINARY\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "SELECT cast('0' as binary) <=> false FROM t" + } ] } @@ -970,7 +1124,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> false)\"", "left" : "\"TIMESTAMP\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> false FROM t" + } ] } @@ -987,5 +1148,12 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> false)\"", "left" : "\"DATE\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 59, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <=> false FROM t" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out index f86740f45b69b..0f485014ef3f7 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out @@ -244,7 +244,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) + cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -261,7 +268,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) + cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -278,7 +292,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) + cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -295,7 +316,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) + cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -312,7 +340,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -329,7 +364,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -346,7 +388,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -363,7 +412,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -672,7 +728,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(3, 0)) + cast('1' as binary) FROM t" + } ] } @@ -689,7 +752,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(5, 0)) + cast('1' as binary) FROM t" + } ] } @@ -706,7 +776,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(10, 0)) + cast('1' as binary) FROM t" + } ] } @@ -723,7 +800,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(20, 0)) + cast('1' as binary) FROM t" + } ] } @@ -740,7 +824,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(3, 0)) + cast(1 as boolean) FROM t" + } ] } @@ -757,7 +848,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(5, 0)) + cast(1 as boolean) FROM t" + } ] } @@ -774,7 +872,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(10, 0)) + cast(1 as boolean) FROM t" + } ] } @@ -791,7 +896,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(20, 0)) + cast(1 as boolean) FROM t" + } ] } @@ -808,7 +920,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -825,7 +944,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -842,7 +968,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -859,7 +992,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -1136,7 +1276,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) - cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -1153,7 +1300,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) - cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -1170,7 +1324,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) - cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -1187,7 +1348,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) - cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -1532,7 +1700,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(3, 0)) - cast('1' as binary) FROM t" + } ] } @@ -1549,7 +1724,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(5, 0)) - cast('1' as binary) FROM t" + } ] } @@ -1566,7 +1748,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(10, 0)) - cast('1' as binary) FROM t" + } ] } @@ -1583,7 +1772,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(20, 0)) - cast('1' as binary) FROM t" + } ] } @@ -1600,7 +1796,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(3, 0)) - cast(1 as boolean) FROM t" + } ] } @@ -1617,7 +1820,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(5, 0)) - cast(1 as boolean) FROM t" + } ] } @@ -1634,7 +1844,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(10, 0)) - cast(1 as boolean) FROM t" + } ] } @@ -1651,7 +1868,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(20, 0)) - cast(1 as boolean) FROM t" + } ] } @@ -1964,7 +2188,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) * cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -1981,7 +2212,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) * cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -1998,7 +2236,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) * cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -2015,7 +2260,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) * cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -2032,7 +2284,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -2049,7 +2308,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -2066,7 +2332,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -2083,7 +2356,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -2100,7 +2380,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -2117,7 +2404,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -2134,7 +2428,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -2151,7 +2452,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -2424,7 +2732,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(3, 0)) * cast('1' as binary) FROM t" + } ] } @@ -2441,7 +2756,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(5, 0)) * cast('1' as binary) FROM t" + } ] } @@ -2458,7 +2780,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(10, 0)) * cast('1' as binary) FROM t" + } ] } @@ -2475,7 +2804,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(20, 0)) * cast('1' as binary) FROM t" + } ] } @@ -2492,7 +2828,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(3, 0)) * cast(1 as boolean) FROM t" + } ] } @@ -2509,7 +2852,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(5, 0)) * cast(1 as boolean) FROM t" + } ] } @@ -2526,7 +2876,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(10, 0)) * cast(1 as boolean) FROM t" + } ] } @@ -2543,7 +2900,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(20, 0)) * cast(1 as boolean) FROM t" + } ] } @@ -2560,7 +2924,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -2577,7 +2948,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -2594,7 +2972,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -2611,7 +2996,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -2628,7 +3020,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00' as date) FROM t" + } ] } @@ -2645,7 +3044,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00' as date) FROM t" + } ] } @@ -2662,7 +3068,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00' as date) FROM t" + } ] } @@ -2679,7 +3092,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00' as date) FROM t" + } ] } @@ -2920,7 +3340,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) / cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -2937,7 +3364,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) / cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -2954,7 +3388,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -2971,7 +3412,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) / cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -2988,7 +3436,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -3005,7 +3460,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -3022,7 +3484,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -3039,7 +3508,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -3056,7 +3532,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -3073,7 +3556,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -3090,7 +3580,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -3107,7 +3604,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -3380,7 +3884,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(3, 0)) / cast('1' as binary) FROM t" + } ] } @@ -3397,7 +3908,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(5, 0)) / cast('1' as binary) FROM t" + } ] } @@ -3414,7 +3932,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t" + } ] } @@ -3431,7 +3956,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(20, 0)) / cast('1' as binary) FROM t" + } ] } @@ -3448,7 +3980,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(3, 0)) / cast(1 as boolean) FROM t" + } ] } @@ -3465,7 +4004,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(5, 0)) / cast(1 as boolean) FROM t" + } ] } @@ -3482,7 +4028,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t" + } ] } @@ -3499,7 +4052,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(20, 0)) / cast(1 as boolean) FROM t" + } ] } @@ -3516,7 +4076,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -3533,7 +4100,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -3550,7 +4124,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -3567,7 +4148,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -3584,7 +4172,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00' as date) FROM t" + } ] } @@ -3601,7 +4196,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00' as date) FROM t" + } ] } @@ -3618,7 +4220,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00' as date) FROM t" + } ] } @@ -3635,7 +4244,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00' as date) FROM t" + } ] } @@ -3876,7 +4492,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) % cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -3893,7 +4516,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) % cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -3910,7 +4540,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) % cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -3927,7 +4564,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) % cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -3944,7 +4588,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -3961,7 +4612,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -3978,7 +4636,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -3995,7 +4660,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -4012,7 +4684,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -4029,7 +4708,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -4046,7 +4732,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -4063,7 +4756,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -4336,7 +5036,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(3, 0)) % cast('1' as binary) FROM t" + } ] } @@ -4353,7 +5060,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(5, 0)) % cast('1' as binary) FROM t" + } ] } @@ -4370,7 +5084,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(10, 0)) % cast('1' as binary) FROM t" + } ] } @@ -4387,7 +5108,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(20, 0)) % cast('1' as binary) FROM t" + } ] } @@ -4404,7 +5132,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(3, 0)) % cast(1 as boolean) FROM t" + } ] } @@ -4421,7 +5156,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(5, 0)) % cast(1 as boolean) FROM t" + } ] } @@ -4438,7 +5180,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(10, 0)) % cast(1 as boolean) FROM t" + } ] } @@ -4455,7 +5204,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(20, 0)) % cast(1 as boolean) FROM t" + } ] } @@ -4472,7 +5228,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -4489,7 +5252,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -4506,7 +5276,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -4523,7 +5300,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -4540,7 +5324,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -4557,7 +5348,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -4574,7 +5372,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -4591,7 +5396,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -4832,7 +5644,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 65, + "fragment" : "SELECT pmod(cast('1' as binary), cast(1 as decimal(3, 0))) FROM t" + } ] } @@ -4849,7 +5668,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 65, + "fragment" : "SELECT pmod(cast('1' as binary), cast(1 as decimal(5, 0))) FROM t" + } ] } @@ -4866,7 +5692,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "SELECT pmod(cast('1' as binary), cast(1 as decimal(10, 0))) FROM t" + } ] } @@ -4883,7 +5716,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "SELECT pmod(cast('1' as binary), cast(1 as decimal(20, 0))) FROM t" + } ] } @@ -4900,7 +5740,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(3, 0))) FROM t" + } ] } @@ -4917,7 +5764,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(5, 0))) FROM t" + } ] } @@ -4934,7 +5788,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 89, + "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(10, 0))) FROM t" + } ] } @@ -4951,7 +5812,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 89, + "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(20, 0))) FROM t" + } ] } @@ -4968,7 +5836,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 81, + "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(3, 0))) FROM t" + } ] } @@ -4985,7 +5860,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 81, + "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(5, 0))) FROM t" + } ] } @@ -5002,7 +5884,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 82, + "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(10, 0))) FROM t" + } ] } @@ -5019,7 +5908,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 82, + "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(20, 0))) FROM t" + } ] } @@ -5292,7 +6188,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "SELECT pmod(cast(1 as decimal(3, 0)) , cast('1' as binary)) FROM t" + } ] } @@ -5309,7 +6212,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "SELECT pmod(cast(1 as decimal(5, 0)) , cast('1' as binary)) FROM t" + } ] } @@ -5326,7 +6236,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "SELECT pmod(cast(1 as decimal(10, 0)), cast('1' as binary)) FROM t" + } ] } @@ -5343,7 +6260,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "SELECT pmod(cast(1 as decimal(20, 0)), cast('1' as binary)) FROM t" + } ] } @@ -5360,7 +6284,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 65, + "fragment" : "SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as boolean)) FROM t" + } ] } @@ -5377,7 +6308,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 65, + "fragment" : "SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as boolean)) FROM t" + } ] } @@ -5394,7 +6332,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 65, + "fragment" : "SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as boolean)) FROM t" + } ] } @@ -5411,7 +6356,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 65, + "fragment" : "SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as boolean)) FROM t" + } ] } @@ -5428,7 +6380,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 89, + "fragment" : "SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t" + } ] } @@ -5445,7 +6404,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 89, + "fragment" : "SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t" + } ] } @@ -5462,7 +6428,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 89, + "fragment" : "SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t" + } ] } @@ -5479,7 +6452,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 89, + "fragment" : "SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t" + } ] } @@ -5496,7 +6476,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 82, + "fragment" : "SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t" + } ] } @@ -5513,7 +6500,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 82, + "fragment" : "SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t" + } ] } @@ -5530,7 +6524,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 82, + "fragment" : "SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t" + } ] } @@ -5547,7 +6548,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 82, + "fragment" : "SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00' as date)) FROM t" + } ] } @@ -5788,7 +6796,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) = cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -5805,7 +6820,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) = cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -5822,7 +6844,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) = cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -5839,7 +6868,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) = cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -5856,7 +6892,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -5873,7 +6916,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -5890,7 +6940,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -5907,7 +6964,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -5924,7 +6988,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -5941,7 +7012,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -5958,7 +7036,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -5975,7 +7060,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -6248,7 +7340,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(3, 0)) = cast('1' as binary) FROM t" + } ] } @@ -6265,7 +7364,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(5, 0)) = cast('1' as binary) FROM t" + } ] } @@ -6282,7 +7388,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(10, 0)) = cast('1' as binary) FROM t" + } ] } @@ -6299,7 +7412,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(20, 0)) = cast('1' as binary) FROM t" + } ] } @@ -6348,7 +7468,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -6365,7 +7492,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -6382,7 +7516,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -6399,7 +7540,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -6416,7 +7564,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -6433,7 +7588,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -6450,7 +7612,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -6467,7 +7636,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -6708,7 +7884,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('1' as binary) <=> cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -6725,7 +7908,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('1' as binary) <=> cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -6742,7 +7932,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "SELECT cast('1' as binary) <=> cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -6759,7 +7956,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "SELECT cast('1' as binary) <=> cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -6776,7 +7980,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -6793,7 +8004,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -6810,7 +8028,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -6827,7 +8052,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -6844,7 +8076,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -6861,7 +8100,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -6878,7 +8124,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 79, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -6895,7 +8148,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 79, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -7168,7 +8428,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "SELECT cast(1 as decimal(3, 0)) <=> cast('1' as binary) FROM t" + } ] } @@ -7185,7 +8452,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "SELECT cast(1 as decimal(5, 0)) <=> cast('1' as binary) FROM t" + } ] } @@ -7202,7 +8476,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "SELECT cast(1 as decimal(10, 0)) <=> cast('1' as binary) FROM t" + } ] } @@ -7219,7 +8500,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "SELECT cast(1 as decimal(20, 0)) <=> cast('1' as binary) FROM t" + } ] } @@ -7268,7 +8556,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -7285,7 +8580,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -7302,7 +8604,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -7319,7 +8628,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -7336,7 +8652,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 79, + "fragment" : "SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -7353,7 +8676,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 79, + "fragment" : "SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -7370,7 +8700,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 79, + "fragment" : "SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -7387,7 +8724,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 79, + "fragment" : "SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -7628,7 +8972,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) < cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -7645,7 +8996,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) < cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -7662,7 +9020,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) < cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -7679,7 +9044,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) < cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -7696,7 +9068,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -7713,7 +9092,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -7730,7 +9116,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -7747,7 +9140,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -7764,7 +9164,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -7781,7 +9188,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -7798,7 +9212,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -7815,7 +9236,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -8088,7 +9516,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(3, 0)) < cast('1' as binary) FROM t" + } ] } @@ -8105,7 +9540,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(5, 0)) < cast('1' as binary) FROM t" + } ] } @@ -8122,7 +9564,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(10, 0)) < cast('1' as binary) FROM t" + } ] } @@ -8139,7 +9588,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(20, 0)) < cast('1' as binary) FROM t" + } ] } @@ -8156,7 +9612,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(3, 0)) < cast(1 as boolean) FROM t" + } ] } @@ -8173,7 +9636,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(5, 0)) < cast(1 as boolean) FROM t" + } ] } @@ -8190,7 +9660,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(10, 0)) < cast(1 as boolean) FROM t" + } ] } @@ -8207,7 +9684,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(20, 0)) < cast(1 as boolean) FROM t" + } ] } @@ -8224,7 +9708,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -8241,7 +9732,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -8258,7 +9756,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -8275,7 +9780,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -8292,7 +9804,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -8309,7 +9828,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -8326,7 +9852,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -8343,7 +9876,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -8584,7 +10124,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) <= cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -8601,7 +10148,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) <= cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -8618,7 +10172,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('1' as binary) <= cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -8635,7 +10196,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('1' as binary) <= cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -8652,7 +10220,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -8669,7 +10244,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -8686,7 +10268,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -8703,7 +10292,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -8720,7 +10316,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -8737,7 +10340,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -8754,7 +10364,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -8771,7 +10388,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -9044,7 +10668,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as decimal(3, 0)) <= cast('1' as binary) FROM t" + } ] } @@ -9061,7 +10692,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as decimal(5, 0)) <= cast('1' as binary) FROM t" + } ] } @@ -9078,7 +10716,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as decimal(10, 0)) <= cast('1' as binary) FROM t" + } ] } @@ -9095,7 +10740,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as decimal(20, 0)) <= cast('1' as binary) FROM t" + } ] } @@ -9112,7 +10764,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(3, 0)) <= cast(1 as boolean) FROM t" + } ] } @@ -9129,7 +10788,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(5, 0)) <= cast(1 as boolean) FROM t" + } ] } @@ -9146,7 +10812,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(10, 0)) <= cast(1 as boolean) FROM t" + } ] } @@ -9163,7 +10836,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(20, 0)) <= cast(1 as boolean) FROM t" + } ] } @@ -9180,7 +10860,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -9197,7 +10884,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -9214,7 +10908,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -9231,7 +10932,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -9248,7 +10956,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -9265,7 +10980,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -9282,7 +11004,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -9299,7 +11028,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -9540,7 +11276,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) > cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -9557,7 +11300,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast('1' as binary) > cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -9574,7 +11324,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) > cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -9591,7 +11348,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) > cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -9608,7 +11372,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -9625,7 +11396,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -9642,7 +11420,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -9659,7 +11444,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -9676,7 +11468,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -9693,7 +11492,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -9710,7 +11516,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -9727,7 +11540,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -10000,7 +11820,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(3, 0)) > cast('1' as binary) FROM t" + } ] } @@ -10017,7 +11844,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(5, 0)) > cast('1' as binary) FROM t" + } ] } @@ -10034,7 +11868,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(10, 0)) > cast('1' as binary) FROM t" + } ] } @@ -10051,7 +11892,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(20, 0)) > cast('1' as binary) FROM t" + } ] } @@ -10068,7 +11916,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(3, 0)) > cast(1 as boolean) FROM t" + } ] } @@ -10085,7 +11940,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(5, 0)) > cast(1 as boolean) FROM t" + } ] } @@ -10102,7 +11964,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(10, 0)) > cast(1 as boolean) FROM t" + } ] } @@ -10119,7 +11988,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(20, 0)) > cast(1 as boolean) FROM t" + } ] } @@ -10136,7 +12012,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -10153,7 +12036,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -10170,7 +12060,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -10187,7 +12084,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -10204,7 +12108,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -10221,7 +12132,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -10238,7 +12156,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -10255,7 +12180,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -10496,7 +12428,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) >= cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -10513,7 +12452,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) >= cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -10530,7 +12476,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('1' as binary) >= cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -10547,7 +12500,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('1' as binary) >= cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -10564,7 +12524,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -10581,7 +12548,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -10598,7 +12572,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -10615,7 +12596,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -10632,7 +12620,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -10649,7 +12644,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -10666,7 +12668,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -10683,7 +12692,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -10956,7 +12972,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as decimal(3, 0)) >= cast('1' as binary) FROM t" + } ] } @@ -10973,7 +12996,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as decimal(5, 0)) >= cast('1' as binary) FROM t" + } ] } @@ -10990,7 +13020,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as decimal(10, 0)) >= cast('1' as binary) FROM t" + } ] } @@ -11007,7 +13044,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as decimal(20, 0)) >= cast('1' as binary) FROM t" + } ] } @@ -11024,7 +13068,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(3, 0)) >= cast(1 as boolean) FROM t" + } ] } @@ -11041,7 +13092,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(5, 0)) >= cast(1 as boolean) FROM t" + } ] } @@ -11058,7 +13116,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(10, 0)) >= cast(1 as boolean) FROM t" + } ] } @@ -11075,7 +13140,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(20, 0)) >= cast(1 as boolean) FROM t" + } ] } @@ -11092,7 +13164,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -11109,7 +13188,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -11126,7 +13212,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -11143,7 +13236,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -11160,7 +13260,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -11177,7 +13284,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -11194,7 +13308,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -11211,7 +13332,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -11452,7 +13580,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) <> cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -11469,7 +13604,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) <> cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -11486,7 +13628,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('1' as binary) <> cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -11503,7 +13652,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('1' as binary) <> cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -11520,7 +13676,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -11537,7 +13700,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -11554,7 +13724,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -11571,7 +13748,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -11588,7 +13772,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(3,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(3,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(3, 0)) FROM t" + } ] } @@ -11605,7 +13796,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(5,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(5,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(5, 0)) FROM t" + } ] } @@ -11622,7 +13820,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -11639,7 +13844,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(20,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(20,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(20, 0)) FROM t" + } ] } @@ -11912,7 +14124,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as decimal(3, 0)) <> cast('1' as binary) FROM t" + } ] } @@ -11929,7 +14148,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as decimal(5, 0)) <> cast('1' as binary) FROM t" + } ] } @@ -11946,7 +14172,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as decimal(10, 0)) <> cast('1' as binary) FROM t" + } ] } @@ -11963,7 +14196,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as decimal(20, 0)) <> cast('1' as binary) FROM t" + } ] } @@ -12012,7 +14252,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -12029,7 +14276,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -12046,7 +14300,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -12063,7 +14324,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -12080,7 +14348,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(3,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -12097,7 +14372,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(5,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -12114,7 +14396,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -12131,5 +14420,12 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(20,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out index 893efa7c57815..e404130cb2510 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out @@ -84,7 +84,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(1 AS BINARY))\"", "left" : "\"TINYINT\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT cast(1 as tinyint) / cast('1' as binary) FROM t" + } ] } @@ -101,7 +108,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(1 AS BOOLEAN))\"", "left" : "\"TINYINT\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 53, + "fragment" : "SELECT cast(1 as tinyint) / cast(1 as boolean) FROM t" + } ] } @@ -118,7 +132,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"TINYINT\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -135,7 +156,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"TINYINT\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -216,7 +244,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(1 AS BINARY))\"", "left" : "\"SMALLINT\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 55, + "fragment" : "SELECT cast(1 as smallint) / cast('1' as binary) FROM t" + } ] } @@ -233,7 +268,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(1 AS BOOLEAN))\"", "left" : "\"SMALLINT\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT cast(1 as smallint) / cast(1 as boolean) FROM t" + } ] } @@ -250,7 +292,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"SMALLINT\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -267,7 +316,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"SMALLINT\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 71, + "fragment" : "SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -348,7 +404,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS INT) / CAST(1 AS BINARY))\"", "left" : "\"INT\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 50, + "fragment" : "SELECT cast(1 as int) / cast('1' as binary) FROM t" + } ] } @@ -365,7 +428,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS INT) / CAST(1 AS BOOLEAN))\"", "left" : "\"INT\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "SELECT cast(1 as int) / cast(1 as boolean) FROM t" + } ] } @@ -382,7 +452,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS INT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"INT\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 73, + "fragment" : "SELECT cast(1 as int) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -399,7 +476,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS INT) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"INT\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "SELECT cast(1 as int) / cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -480,7 +564,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(1 AS BINARY))\"", "left" : "\"BIGINT\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 53, + "fragment" : "SELECT cast(1 as bigint) / cast('1' as binary) FROM t" + } ] } @@ -497,7 +588,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(1 AS BOOLEAN))\"", "left" : "\"BIGINT\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 52, + "fragment" : "SELECT cast(1 as bigint) / cast(1 as boolean) FROM t" + } ] } @@ -514,7 +612,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"BIGINT\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -531,7 +636,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"BIGINT\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 69, + "fragment" : "SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -612,7 +724,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(1 AS BINARY))\"", "left" : "\"FLOAT\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 52, + "fragment" : "SELECT cast(1 as float) / cast('1' as binary) FROM t" + } ] } @@ -629,7 +748,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(1 AS BOOLEAN))\"", "left" : "\"FLOAT\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 51, + "fragment" : "SELECT cast(1 as float) / cast(1 as boolean) FROM t" + } ] } @@ -646,7 +772,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"FLOAT\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 75, + "fragment" : "SELECT cast(1 as float) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -663,7 +796,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"FLOAT\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 68, + "fragment" : "SELECT cast(1 as float) / cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -744,7 +884,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 53, + "fragment" : "SELECT cast(1 as double) / cast('1' as binary) FROM t" + } ] } @@ -761,7 +908,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 52, + "fragment" : "SELECT cast(1 as double) / cast(1 as boolean) FROM t" + } ] } @@ -778,7 +932,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DOUBLE\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast(1 as double) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -795,7 +956,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DOUBLE\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 69, + "fragment" : "SELECT cast(1 as double) / cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -876,7 +1044,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BINARY))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t" + } ] } @@ -893,7 +1068,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t" + } ] } @@ -910,7 +1092,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -927,7 +1116,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DECIMAL(10,0)\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -1008,7 +1204,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 53, + "fragment" : "SELECT cast(1 as string) / cast('1' as binary) FROM t" + } ] } @@ -1025,7 +1228,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 52, + "fragment" : "SELECT cast(1 as string) / cast(1 as boolean) FROM t" + } ] } @@ -1042,7 +1252,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DOUBLE\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast(1 as string) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -1059,7 +1276,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DOUBLE\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 69, + "fragment" : "SELECT cast(1 as string) / cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -1076,7 +1300,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS TINYINT))\"", "left" : "\"BINARY\"", "right" : "\"TINYINT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT cast('1' as binary) / cast(1 as tinyint) FROM t" + } ] } @@ -1093,7 +1324,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS SMALLINT))\"", "left" : "\"BINARY\"", "right" : "\"SMALLINT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 55, + "fragment" : "SELECT cast('1' as binary) / cast(1 as smallint) FROM t" + } ] } @@ -1110,7 +1348,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS INT))\"", "left" : "\"BINARY\"", "right" : "\"INT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 50, + "fragment" : "SELECT cast('1' as binary) / cast(1 as int) FROM t" + } ] } @@ -1127,7 +1372,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS BIGINT))\"", "left" : "\"BINARY\"", "right" : "\"BIGINT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 53, + "fragment" : "SELECT cast('1' as binary) / cast(1 as bigint) FROM t" + } ] } @@ -1144,7 +1396,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS FLOAT))\"", "left" : "\"BINARY\"", "right" : "\"FLOAT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 52, + "fragment" : "SELECT cast('1' as binary) / cast(1 as float) FROM t" + } ] } @@ -1161,7 +1420,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DOUBLE))\"", "left" : "\"BINARY\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 53, + "fragment" : "SELECT cast('1' as binary) / cast(1 as double) FROM t" + } ] } @@ -1178,7 +1444,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BINARY\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -1195,7 +1468,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS STRING))\"", "left" : "\"BINARY\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 53, + "fragment" : "SELECT cast('1' as binary) / cast(1 as string) FROM t" + } ] } @@ -1212,7 +1492,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS BINARY))\"", "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "actualDataType" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 55, + "fragment" : "SELECT cast('1' as binary) / cast('1' as binary) FROM t" + } ] } @@ -1229,7 +1516,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS BOOLEAN))\"", "left" : "\"BINARY\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT cast('1' as binary) / cast(1 as boolean) FROM t" + } ] } @@ -1246,7 +1540,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"BINARY\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast('1' as binary) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -1263,7 +1564,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"BINARY\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 71, + "fragment" : "SELECT cast('1' as binary) / cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -1280,7 +1588,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS TINYINT))\"", "left" : "\"BOOLEAN\"", "right" : "\"TINYINT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 53, + "fragment" : "SELECT cast(1 as boolean) / cast(1 as tinyint) FROM t" + } ] } @@ -1297,7 +1612,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS SMALLINT))\"", "left" : "\"BOOLEAN\"", "right" : "\"SMALLINT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT cast(1 as boolean) / cast(1 as smallint) FROM t" + } ] } @@ -1314,7 +1636,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS INT))\"", "left" : "\"BOOLEAN\"", "right" : "\"INT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "SELECT cast(1 as boolean) / cast(1 as int) FROM t" + } ] } @@ -1331,7 +1660,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS BIGINT))\"", "left" : "\"BOOLEAN\"", "right" : "\"BIGINT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 52, + "fragment" : "SELECT cast(1 as boolean) / cast(1 as bigint) FROM t" + } ] } @@ -1348,7 +1684,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS FLOAT))\"", "left" : "\"BOOLEAN\"", "right" : "\"FLOAT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 51, + "fragment" : "SELECT cast(1 as boolean) / cast(1 as float) FROM t" + } ] } @@ -1365,7 +1708,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS DOUBLE))\"", "left" : "\"BOOLEAN\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 52, + "fragment" : "SELECT cast(1 as boolean) / cast(1 as double) FROM t" + } ] } @@ -1382,7 +1732,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"BOOLEAN\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 60, + "fragment" : "SELECT cast(1 as boolean) / cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -1399,7 +1756,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS STRING))\"", "left" : "\"BOOLEAN\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 52, + "fragment" : "SELECT cast(1 as boolean) / cast(1 as string) FROM t" + } ] } @@ -1416,7 +1780,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS BINARY))\"", "left" : "\"BOOLEAN\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT cast(1 as boolean) / cast('1' as binary) FROM t" + } ] } @@ -1433,7 +1804,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS BOOLEAN))\"", "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "actualDataType" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 53, + "fragment" : "SELECT cast(1 as boolean) / cast(1 as boolean) FROM t" + } ] } @@ -1450,7 +1828,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"BOOLEAN\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -1467,7 +1852,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"BOOLEAN\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -1484,7 +1876,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS TINYINT))\"", "left" : "\"TIMESTAMP\"", "right" : "\"TINYINT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as tinyint) FROM t" + } ] } @@ -1501,7 +1900,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS SMALLINT))\"", "left" : "\"TIMESTAMP\"", "right" : "\"SMALLINT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as smallint) FROM t" + } ] } @@ -1518,7 +1924,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS INT))\"", "left" : "\"TIMESTAMP\"", "right" : "\"INT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 73, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as int) FROM t" + } ] } @@ -1535,7 +1948,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS BIGINT))\"", "left" : "\"TIMESTAMP\"", "right" : "\"BIGINT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as bigint) FROM t" + } ] } @@ -1552,7 +1972,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS FLOAT))\"", "left" : "\"TIMESTAMP\"", "right" : "\"FLOAT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 75, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as float) FROM t" + } ] } @@ -1569,7 +1996,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DOUBLE))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as double) FROM t" + } ] } @@ -1586,7 +2020,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -1603,7 +2044,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS STRING))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 76, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as string) FROM t" + } ] } @@ -1620,7 +2068,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS BINARY))\"", "left" : "\"TIMESTAMP\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('1' as binary) FROM t" + } ] } @@ -1637,7 +2092,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS BOOLEAN))\"", "left" : "\"TIMESTAMP\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as boolean) FROM t" + } ] } @@ -1654,7 +2116,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "actualDataType" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 101, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -1671,7 +2140,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"TIMESTAMP\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 94, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -1688,7 +2164,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS TINYINT))\"", "left" : "\"DATE\"", "right" : "\"TINYINT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as tinyint) FROM t" + } ] } @@ -1705,7 +2188,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS SMALLINT))\"", "left" : "\"DATE\"", "right" : "\"SMALLINT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 71, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as smallint) FROM t" + } ] } @@ -1722,7 +2212,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS INT))\"", "left" : "\"DATE\"", "right" : "\"INT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as int) FROM t" + } ] } @@ -1739,7 +2236,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS BIGINT))\"", "left" : "\"DATE\"", "right" : "\"BIGINT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 69, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as bigint) FROM t" + } ] } @@ -1756,7 +2260,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS FLOAT))\"", "left" : "\"DATE\"", "right" : "\"FLOAT\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 68, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as float) FROM t" + } ] } @@ -1773,7 +2284,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS DOUBLE))\"", "left" : "\"DATE\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 69, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as double) FROM t" + } ] } @@ -1790,7 +2308,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(10,0)))\"", "left" : "\"DATE\"", "right" : "\"DECIMAL(10,0)\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t" + } ] } @@ -1807,7 +2332,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS STRING))\"", "left" : "\"DATE\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 69, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as string) FROM t" + } ] } @@ -1824,7 +2356,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS BINARY))\"", "left" : "\"DATE\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 71, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast('1' as binary) FROM t" + } ] } @@ -1841,7 +2380,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS BOOLEAN))\"", "left" : "\"DATE\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as boolean) FROM t" + } ] } @@ -1858,7 +2404,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DATE\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 94, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -1875,5 +2428,12 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(2017-12-11 09:30:00 AS DATE))\"", "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "actualDataType" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00' as date) FROM t" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out index b866d457196c6..53a7269322a5a 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out @@ -84,7 +84,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 + CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' + cast('1' as binary) FROM t" + } ] } @@ -101,7 +108,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 + CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' + cast(1 as boolean) FROM t" + } ] } @@ -118,7 +132,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DOUBLE\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' + cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -208,7 +229,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 - CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' - cast('1' as binary) FROM t" + } ] } @@ -225,7 +253,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 - CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' - cast(1 as boolean) FROM t" + } ] } @@ -323,7 +358,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 * CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' * cast('1' as binary) FROM t" + } ] } @@ -340,7 +382,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 * CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' * cast(1 as boolean) FROM t" + } ] } @@ -357,7 +406,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 * CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DOUBLE\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' * cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -374,7 +430,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 * CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DOUBLE\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' * cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -455,7 +518,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 / CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' / cast('1' as binary) FROM t" + } ] } @@ -472,7 +542,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 / CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' / cast(1 as boolean) FROM t" + } ] } @@ -489,7 +566,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DOUBLE\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' / cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -506,7 +590,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 / CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DOUBLE\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' / cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -587,7 +678,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 % CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' % cast('1' as binary) FROM t" + } ] } @@ -604,7 +702,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 % CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' % cast(1 as boolean) FROM t" + } ] } @@ -621,7 +726,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DOUBLE\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' % cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] } @@ -638,7 +750,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(1 % CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DOUBLE\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT '1' % cast('2017-12-11 09:30:00' as date) FROM t" + } ] } @@ -719,7 +838,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(1, CAST(1 AS BINARY))\"", "left" : "\"DOUBLE\"", "right" : "\"BINARY\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 67, + "fragment" : "SELECT pmod('1', cast('1' as binary)) FROM t" + } ] } @@ -736,7 +862,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(1, CAST(1 AS BOOLEAN))\"", "left" : "\"DOUBLE\"", "right" : "\"BOOLEAN\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 67, + "fragment" : "SELECT pmod('1', cast(1 as boolean)) FROM t" + } ] } @@ -753,7 +886,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(1, CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"", "left" : "\"DOUBLE\"", "right" : "\"TIMESTAMP\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 67, + "fragment" : "SELECT pmod('1', cast('2017-12-11 09:30:00.0' as timestamp)) FROM t" + } ] } @@ -770,7 +910,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(1, CAST(2017-12-11 09:30:00 AS DATE))\"", "left" : "\"DOUBLE\"", "right" : "\"DATE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 67, + "fragment" : "SELECT pmod('1', cast('2017-12-11 09:30:00' as date)) FROM t" + } ] } @@ -843,7 +990,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) + 1)\"", "left" : "\"BINARY\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('1' as binary) + '1' FROM t" + } ] } @@ -860,7 +1014,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) + 1)\"", "left" : "\"BOOLEAN\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as boolean) + '1' FROM t" + } ] } @@ -877,7 +1038,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + 1)\"", "left" : "\"TIMESTAMP\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) + '1' FROM t" + } ] } @@ -959,7 +1127,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) - 1)\"", "left" : "\"BINARY\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('1' as binary) - '1' FROM t" + } ] } @@ -976,7 +1151,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) - 1)\"", "left" : "\"BOOLEAN\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as boolean) - '1' FROM t" + } ] } @@ -1067,7 +1249,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) * 1)\"", "left" : "\"BINARY\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('1' as binary) * '1' FROM t" + } ] } @@ -1084,7 +1273,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) * 1)\"", "left" : "\"BOOLEAN\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as boolean) * '1' FROM t" + } ] } @@ -1101,7 +1297,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) * 1)\"", "left" : "\"TIMESTAMP\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) * '1' FROM t" + } ] } @@ -1118,7 +1321,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) * 1)\"", "left" : "\"DATE\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) * '1' FROM t" + } ] } @@ -1191,7 +1401,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) / 1)\"", "left" : "\"BINARY\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('1' as binary) / '1' FROM t" + } ] } @@ -1208,7 +1425,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / 1)\"", "left" : "\"BOOLEAN\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as boolean) / '1' FROM t" + } ] } @@ -1225,7 +1449,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / 1)\"", "left" : "\"TIMESTAMP\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) / '1' FROM t" + } ] } @@ -1242,7 +1473,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / 1)\"", "left" : "\"DATE\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) / '1' FROM t" + } ] } @@ -1315,7 +1553,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BINARY) % 1)\"", "left" : "\"BINARY\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('1' as binary) % '1' FROM t" + } ] } @@ -1332,7 +1577,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(1 AS BOOLEAN) % 1)\"", "left" : "\"BOOLEAN\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast(1 as boolean) % '1' FROM t" + } ] } @@ -1349,7 +1601,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % 1)\"", "left" : "\"TIMESTAMP\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('2017-12-11 09:30:00.0' as timestamp) % '1' FROM t" + } ] } @@ -1366,7 +1625,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % 1)\"", "left" : "\"DATE\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT cast('2017-12-11 09:30:00' as date) % '1' FROM t" + } ] } @@ -1439,7 +1705,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS BINARY), 1)\"", "left" : "\"BINARY\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 67, + "fragment" : "SELECT pmod(cast('1' as binary), '1') FROM t" + } ] } @@ -1456,7 +1729,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(1 AS BOOLEAN), 1)\"", "left" : "\"BOOLEAN\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 67, + "fragment" : "SELECT pmod(cast(1 as boolean), '1') FROM t" + } ] } @@ -1473,7 +1753,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), 1)\"", "left" : "\"TIMESTAMP\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 67, + "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), '1') FROM t" + } ] } @@ -1490,7 +1777,14 @@ org.apache.spark.sql.AnalysisException "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), 1)\"", "left" : "\"DATE\"", "right" : "\"DOUBLE\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 67, + "fragment" : "SELECT pmod(cast('2017-12-11 09:30:00' as date), '1') FROM t" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out index fc11234112b40..4cc2ca9fdecf0 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out @@ -493,5 +493,12 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`o`.`unique1`", "proposal" : "`i`.`unique1`, `i`.`unique2`, `i`.`hundred`, `i`.`even`, `i`.`four`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 100, + "fragment" : "select\n (select udf(max((select i.unique2 from tenk1 i where i.unique1 = o.unique1))))\nfrom tenk1 o" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out index 328f4de9aa1ee..45f1a631b4597 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out @@ -3280,7 +3280,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`y`.`f1`", "proposal" : "`j`.`f1`, `j`.`f1`, `x`.`q1`, `x`.`q2`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "select * from\n int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(y.f1)" + } ] } @@ -3307,7 +3314,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`t1`.`uunique1`", "proposal" : "`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`hundred`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 77, + "fragment" : "select udf(t1.uunique1) from\n tenk1 t1 join tenk2 t2 on t1.two = udf(t2.two)" + } ] } @@ -3325,7 +3339,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`t2`.`uunique1`", "proposal" : "`t2`.`unique1`, `t1`.`unique1`, `t2`.`unique2`, `t1`.`unique2`, `t2`.`hundred`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 82, + "fragment" : "select udf(udf(t2.uunique1)) from\n tenk1 t1 join tenk2 t2 on udf(t1.two) = t2.two" + } ] } @@ -3343,7 +3364,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`uunique1`", "proposal" : "`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`even`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 79, + "fragment" : "select udf(uunique1) from\n tenk1 t1 join tenk2 t2 on udf(t1.two) = udf(t2.two)" + } ] } @@ -3550,7 +3578,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "objectName" : "`f1`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 68, + "fragment" : "select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss" + } ] } @@ -3566,7 +3601,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "objectName" : "`a`.`f1`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 55, + "fragment" : "select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss" + } ] } @@ -3582,7 +3624,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "objectName" : "`f1`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 73, + "fragment" : "select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss" + } ] } @@ -3598,7 +3647,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "objectName" : "`a`.`f1`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 75, + "fragment" : "select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out index f46dd188ec435..d6c39b73df994 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out @@ -156,7 +156,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`a`", "proposal" : "`one`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 50, + "fragment" : "SELECT 1 AS one FROM test_having HAVING udf(a) > 1" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out index fa7693046b8a7..804f7287437b2 100755 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out @@ -129,7 +129,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`b`", "proposal" : "`udf(count(1))`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 63, + "stopIndex" : 77, + "fragment" : "ORDER BY udf(b)" + } ] } @@ -217,7 +224,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "index" : "3", "size" : "2" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 64, + "stopIndex" : 64, + "fragment" : "3" + } ] } @@ -349,7 +363,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`b`", "proposal" : "`udf(count(udf(a)))`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 68, + "stopIndex" : 82, + "fragment" : "ORDER BY udf(b)" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index 8e8ef78698cf6..6e5f9c4ff9946 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -48,7 +48,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"a\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 49, + "stopIndex" : 58, + "fragment" : "GROUP BY b" + } ] } @@ -118,7 +125,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"a\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 48, + "stopIndex" : 61, + "fragment" : "GROUP BY a + 1" + } ] } @@ -199,7 +213,14 @@ org.apache.spark.sql.AnalysisException "sqlState" : "42000", "messageParameters" : { "expression" : "\"k\"" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 69, + "stopIndex" : 83, + "fragment" : "GROUP BY udf(a)" + } ] } @@ -224,7 +245,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`k`", "proposal" : "`testdata`.`a`, `testdata`.`b`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 49, + "stopIndex" : 58, + "fragment" : "GROUP BY k" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out index dec5e2b526811..dad37991f78af 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out @@ -236,7 +236,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`year`", "proposal" : "`__auto_generated_subquery_name`.`course`, `__auto_generated_subquery_name`.`earnings`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 118, + "fragment" : "SELECT * FROM (\n SELECT course, earnings FROM courseSales\n)\nPIVOT (\n udf(sum(earnings))\n FOR year IN (2012, 2013)\n)" + } ] } @@ -346,7 +353,14 @@ org.apache.spark.sql.AnalysisException "messageParameters" : { "objectName" : "`s`", "proposal" : "`coursesales`.`year`, `coursesales`.`course`, `coursesales`.`earnings`" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 80, + "fragment" : "SELECT * FROM courseSales\nPIVOT (\n udf(sum(earnings))\n FOR year IN (s, 2013)\n)" + } ] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index e1f3cc060c86a..7e0d89b0fd23b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} import org.apache.spark.sql.catalyst.optimizer.TransposeWindow import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +import org.apache.spark.sql.errors.QueryErrorsSuiteBase import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, Exchange, ShuffleExchangeExec} import org.apache.spark.sql.execution.window.WindowExec @@ -37,6 +38,7 @@ import org.apache.spark.sql.types._ */ class DataFrameWindowFunctionsSuite extends QueryTest with SharedSparkSession + with QueryErrorsSuiteBase with AdaptiveSparkPlanHelper { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala index 6d1b4eaf36b47..7f812e7709efa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.Hex import org.apache.spark.sql.connector.catalog.InMemoryPartitionTableCatalog +import org.apache.spark.sql.errors.QueryErrorsSuiteBase import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.unsafe.types.UTF8String @@ -27,7 +28,7 @@ import org.apache.spark.unsafe.types.UTF8String /** * The base trait for SQL INSERT. */ -trait SQLInsertTestSuite extends QueryTest with SQLTestUtils { +trait SQLInsertTestSuite extends QueryTest with SQLTestUtils with QueryErrorsSuiteBase { import testImplicits._ @@ -169,8 +170,12 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils { exception = intercept[AnalysisException](sql(s"INSERT INTO t1 (c1, c2, c4) values(1, 2, 3)")), errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), - parameters = Map("objectName" -> "`c4`", "proposal" -> "`c1`, `c2`, `c3`")) + errorSubClass = "WITH_SUGGESTION", + sqlState = None, + parameters = Map("objectName" -> "`c4`", "proposal" -> "`c1`, `c2`, `c3`"), + context = ExpectedContext( + fragment = "INSERT INTO t1 (c1, c2, c4)", start = 0, stop = 26 + )) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 13994baf2d052..4ac294276c073 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{Complete, Partial} import org.apache.spark.sql.catalyst.optimizer.{ConvertToLocalRelation, NestedColumnAliasingSuite} import org.apache.spark.sql.catalyst.plans.logical.{LocalLimit, Project, RepartitionByExpression, Sort} import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME +import org.apache.spark.sql.errors.QueryErrorsSuiteBase import org.apache.spark.sql.execution.{CommandResultExec, UnionExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate._ @@ -58,7 +59,7 @@ import org.apache.spark.util.ResetSystemProperties @ExtendedSQLTest class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlanHelper - with ResetSystemProperties { + with ResetSystemProperties with QueryErrorsSuiteBase { import testImplicits._ setupTestData() @@ -2663,16 +2664,22 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark sql("CREATE TABLE t(c struct) USING parquet") sql("CREATE TABLE S(C struct) USING parquet") checkAnswer(sql("SELECT * FROM t, S WHERE t.c.f = S.C.F"), Seq.empty) + val query = "SELECT * FROM t, S WHERE c = C" checkError( exception = intercept[AnalysisException] { - sql("SELECT * FROM t, S WHERE c = C") + sql(query) }, errorClass = "DATATYPE_MISMATCH", - errorSubClass = Some("BINARY_OP_DIFF_TYPES"), + errorSubClass = "BINARY_OP_DIFF_TYPES", + sqlState = None, parameters = Map( "sqlExpr" -> "\"(c = C)\"", "left" -> "\"STRUCT\"", - "right" -> "\"STRUCT\"")) + "right" -> "\"STRUCT\""), + context = ExpectedContext( + fragment = query, + start = 0, + stop = 29)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index a0ccfb10ca8ac..36fcd042e05ec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project, Sort} +import org.apache.spark.sql.errors.QueryErrorsSuiteBase import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecution} import org.apache.spark.sql.execution.datasources.FileScanRDD @@ -29,7 +30,10 @@ import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession -class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlanHelper { +class SubquerySuite extends QueryTest + with SharedSparkSession + with QueryErrorsSuiteBase + with AdaptiveSparkPlanHelper { import testImplicits._ setupTestData() @@ -886,14 +890,18 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark test("SPARK-20688: correctly check analysis for scalar sub-queries") { withTempView("t") { Seq(1 -> "a").toDF("i", "j").createOrReplaceTempView("t") + val query = "SELECT (SELECT count(*) FROM t WHERE a = 1)" checkError( exception = - intercept[AnalysisException](sql("SELECT (SELECT count(*) FROM t WHERE a = 1)")), + intercept[AnalysisException](sql(query)), errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), + errorSubClass = "WITH_SUGGESTION", + sqlState = None, parameters = Map( "objectName" -> "`a`", - "proposal" -> "`t`.`i`, `t`.`j`")) + "proposal" -> "`t`.`i`, `t`.`j`"), + context = ExpectedContext( + fragment = query, start = 0, stop = 42)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala index 595d632512f3f..4258a0296cf8c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.api.java.{UDF1, UDF2, UDF23Test} import org.apache.spark.sql.expressions.SparkUserDefinedFunction import org.apache.spark.sql.functions.{grouping, grouping_id, lit, struct, sum, udf} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, MapType, StringType, StructField, StructType} case class StringLongClass(a: String, b: Long) @@ -34,7 +35,8 @@ case class ArrayClass(arr: Seq[StringIntClass]) class QueryCompilationErrorsSuite extends QueryTest - with QueryErrorsSuiteBase { + with QueryErrorsSuiteBase + with SharedSparkSession { import testImplicits._ test("CANNOT_UP_CAST_DATATYPE: invalid upcast data type") { @@ -119,7 +121,9 @@ class QueryCompilationErrorsSuite parameters = Map( "parameter" -> "strfmt", "functionName" -> "`format_string`", - "expected" -> "expects %1$, %2$ and so on, but got %0$.")) + "expected" -> "expects %1$, %2$ and so on, but got %0$."), + context = ExpectedContext( + fragment = "format_string('%0$s', 'Hello')", start = 7, stop = 36)) } } @@ -213,7 +217,9 @@ class QueryCompilationErrorsSuite checkError( exception = e, errorClass = "NO_HANDLER_FOR_UDAF", - parameters = Map("functionName" -> "org.apache.spark.sql.errors.MyCastToString")) + parameters = Map("functionName" -> "org.apache.spark.sql.errors.MyCastToString"), + context = ExpectedContext( + fragment = "myCast(123)", start = 7, stop = 17)) } } @@ -402,15 +408,17 @@ class QueryCompilationErrorsSuite test("UNRESOLVED_MAP_KEY: string type literal should be quoted") { checkAnswer(sql("select m['a'] from (select map('a', 'b') as m, 'aa' as aa)"), Row("b")) + val query = "select m[a] from (select map('a', 'b') as m, 'aa' as aa)" checkError( - exception = intercept[AnalysisException] { - sql("select m[a] from (select map('a', 'b') as m, 'aa' as aa)") - }, + exception = intercept[AnalysisException] {sql(query)}, errorClass = "UNRESOLVED_MAP_KEY", - errorSubClass = Some("WITH_SUGGESTION"), + errorSubClass = "WITH_SUGGESTION", + sqlState = None, parameters = Map("columnName" -> "`a`", "proposal" -> - "`__auto_generated_subquery_name`.`m`, `__auto_generated_subquery_name`.`aa`")) + "`__auto_generated_subquery_name`.`m`, `__auto_generated_subquery_name`.`aa`"), + context = ExpectedContext( + fragment = query, start = 0, stop = 55)) } test("UNRESOLVED_COLUMN: SELECT distinct does not work correctly " + @@ -437,11 +445,14 @@ class QueryCompilationErrorsSuite |""".stripMargin) }, errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), + errorSubClass = "WITH_SUGGESTION", + sqlState = None, parameters = Map( "objectName" -> "`struct`.`a`", "proposal" -> "`a`, `b`" - ) + ), + context = ExpectedContext( + fragment = "order by struct.a, struct.b", start = 171, stop = 197) ) } @@ -450,13 +461,17 @@ class QueryCompilationErrorsSuite Seq(1 -> "a").toDF("i", "j").createOrReplaceTempView("v") checkAnswer(sql("SELECT i from (SELECT i FROM v)"), Row(1)) + val query = "SELECT v.i from (SELECT i FROM v)" checkError( - exception = intercept[AnalysisException](sql("SELECT v.i from (SELECT i FROM v)")), + exception = intercept[AnalysisException](sql(query)), errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), + errorSubClass = "WITH_SUGGESTION", + sqlState = None, parameters = Map( "objectName" -> "`v`.`i`", - "proposal" -> "`__auto_generated_subquery_name`.`i`")) + "proposal" -> "`__auto_generated_subquery_name`.`i`"), + context = ExpectedContext( + fragment = query, start = 0, stop = 32)) checkAnswer(sql("SELECT __auto_generated_subquery_name.i from (SELECT i FROM v)"), Row(1)) } @@ -468,12 +483,15 @@ class QueryCompilationErrorsSuite sql("CREATE TABLE t(c struct) USING parquet") } + val query = "ALTER TABLE t CHANGE COLUMN c.X COMMENT 'new comment'" checkError( exception = intercept[AnalysisException] { - sql("ALTER TABLE t CHANGE COLUMN c.X COMMENT 'new comment'") + sql(query) }, errorClass = "AMBIGUOUS_COLUMN_OR_FIELD", - parameters = Map("name" -> "`c`.`X`", "n" -> "2")) + parameters = Map("name" -> "`c`.`X`", "n" -> "2"), + context = ExpectedContext( + fragment = query, start = 0, stop = 52)) } } @@ -510,7 +528,9 @@ class QueryCompilationErrorsSuite checkError( exception = e, errorClass = "INVALID_FIELD_NAME", - parameters = Map("fieldName" -> "`m`.`n`", "path" -> "`m`")) + parameters = Map("fieldName" -> "`m`.`n`", "path" -> "`m`"), + context = ExpectedContext( + fragment = "m.n int", start = 27, stop = 33)) } } @@ -619,10 +639,14 @@ class QueryCompilationErrorsSuite checkError( exception = e, errorClass = "UNSUPPORTED_GENERATOR", - errorSubClass = Some("NOT_GENERATOR"), + errorSubClass = "NOT_GENERATOR", + sqlState = None, parameters = Map( "functionName" -> "`array_contains`", - "classCanonicalName" -> "org.apache.spark.sql.catalyst.expressions.ArrayContains")) + "classCanonicalName" -> "org.apache.spark.sql.catalyst.expressions.ArrayContains"), + context = ExpectedContext( + fragment = "LATERAL VIEW array_contains(value, 1) AS explodedvalue", + start = 62, stop = 115)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryErrorsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryErrorsSuiteBase.scala index 02463f9e65a0e..bf1c2bb812d6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryErrorsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryErrorsSuiteBase.scala @@ -18,9 +18,8 @@ package org.apache.spark.sql.errors import org.apache.spark.QueryContext -import org.apache.spark.sql.test.SharedSparkSession -trait QueryErrorsSuiteBase extends SharedSparkSession { +trait QueryErrorsSuiteBase { case class ExpectedContext( objectType: String, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala index 292c52f6dcb60..a2728c4886ff4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionAnsiErrorsSuite.scala @@ -20,10 +20,14 @@ import org.apache.spark._ import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.expressions.{Cast, CheckOverflowInTableInsert, Literal} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.ByteType // Test suite for all the execution errors that requires enable ANSI SQL mode. -class QueryExecutionAnsiErrorsSuite extends QueryTest with QueryErrorsSuiteBase { +class QueryExecutionAnsiErrorsSuite extends QueryTest + with SharedSparkSession + with QueryErrorsSuiteBase { + override def sparkConf: SparkConf = super.sparkConf.set(SQLConf.ANSI_ENABLED.key, "true") private val ansiConf = "\"" + SQLConf.ANSI_ENABLED.key + "\"" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index 1b5fa2aa890c4..bb415ba6afc0e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -42,6 +42,7 @@ import org.apache.spark.sql.functions.{lit, lower, struct, sum, udf} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy.EXCEPTION import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects} +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{DataType, DecimalType, LongType, MetadataBuilder, StructType} import org.apache.spark.util.Utils @@ -49,6 +50,7 @@ class QueryExecutionErrorsSuite extends QueryTest with ParquetTest with OrcTest + with SharedSparkSession with QueryErrorsSuiteBase { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala index 96ebf3ccf0b44..883b581d91b66 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala @@ -20,10 +20,11 @@ package org.apache.spark.sql.errors import org.apache.spark.SparkThrowable import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.test.SharedSparkSession // Turn of the length check because most of the tests check entire error messages // scalastyle:off line.size.limit -class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase { +class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase with SharedSparkSession { private def parseException(sqlText: String): SparkThrowable = { intercept[ParseException](sql(sqlText).collect()) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index fe422cefee032..0689dadac7ba9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME +import org.apache.spark.sql.errors.QueryErrorsSuiteBase import org.apache.spark.sql.internal.SQLConf._ import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} @@ -33,7 +34,7 @@ class SimpleSQLViewSuite extends SQLViewSuite with SharedSparkSession /** * A suite for testing view related functionality. */ -abstract class SQLViewSuite extends QueryTest with SQLTestUtils { +abstract class SQLViewSuite extends QueryTest with SQLTestUtils with QueryErrorsSuiteBase { import testImplicits._ protected override def beforeAll(): Unit = { @@ -886,10 +887,18 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { } checkError(e, errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), + errorSubClass = "WITH_SUGGESTION", + sqlState = None, parameters = Map( "objectName" -> "`C1`", - "proposal" -> "`spark_catalog`.`default`.`t`.`c1`")) + "proposal" -> "`spark_catalog`.`default`.`t`.`c1`"), + context = ExpectedContext( + objectType = "VIEW", + objectName = "spark_catalog.default.v1", + startIndex = 7, + stopIndex = 8, + fragment = "C1" + )) } withSQLConf(ORDER_BY_ORDINAL.key -> "false") { checkAnswer(sql("SELECT * FROM v2"), Seq(Row(3), Row(2), Row(1))) @@ -909,10 +918,18 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { } checkError(e, errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), + errorSubClass = "WITH_SUGGESTION", + sqlState = None, parameters = Map( "objectName" -> "`a`", - "proposal" -> "`spark_catalog`.`default`.`t`.`c1`")) + "proposal" -> "`spark_catalog`.`default`.`t`.`c1`"), + context = ExpectedContext( + objectType = "VIEW", + objectName = "spark_catalog.default.v4", + startIndex = 49, + stopIndex = 49, + fragment = "a" + )) } withSQLConf(ANSI_ENABLED.key -> "true") { val e = intercept[ArithmeticException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala index 2f2a55642e15d..93d210a47f9d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.command.v2 import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.connector.catalog.TableCatalog +import org.apache.spark.sql.errors.QueryErrorsSuiteBase import org.apache.spark.sql.execution.command import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StringType @@ -27,7 +28,9 @@ import org.apache.spark.util.Utils /** * The class contains tests for the `DESCRIBE TABLE` command to check V2 table catalogs. */ -class DescribeTableSuite extends command.DescribeTableSuiteBase with CommandSuiteBase { +class DescribeTableSuite extends command.DescribeTableSuiteBase + with CommandSuiteBase + with QueryErrorsSuiteBase { test("Describing a partition is not supported") { withNamespaceAndTable("ns", "table") { tbl => @@ -99,16 +102,19 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase with CommandSuit |CREATE TABLE $tbl |(key int COMMENT 'column_comment', col struct) |$defaultUsing""".stripMargin) + val query = s"DESC $tbl key1" checkError( exception = intercept[AnalysisException] { - sql(s"DESC $tbl key1").collect() + sql(query).collect() }, errorClass = "UNRESOLVED_COLUMN", errorSubClass = "WITH_SUGGESTION", sqlState = "42000", parameters = Map( "objectName" -> "`key1`", - "proposal" -> "`test_catalog`.`ns`.`tbl`.`key`, `test_catalog`.`ns`.`tbl`.`col`")) + "proposal" -> "`test_catalog`.`ns`.`tbl`.`key`, `test_catalog`.`ns`.`tbl`.`col`"), + context = ExpectedContext( + fragment = query, start = 0, stop = 28)) } } @@ -125,16 +131,19 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase with CommandSuit withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { withNamespaceAndTable("ns", "tbl") { tbl => sql(s"CREATE TABLE $tbl (key int COMMENT 'comment1') $defaultUsing") + val query = s"DESC $tbl KEY" checkError( exception = intercept[AnalysisException] { - sql(s"DESC $tbl KEY").collect() + sql(query).collect() }, errorClass = "UNRESOLVED_COLUMN", errorSubClass = "WITH_SUGGESTION", sqlState = "42000", parameters = Map( "objectName" -> "`KEY`", - "proposal" -> "`test_catalog`.`ns`.`tbl`.`key`")) + "proposal" -> "`test_catalog`.`ns`.`tbl`.`key`"), + context = ExpectedContext( + fragment = query, start = 0, stop = 27)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index bcd485a129c92..735d3f1580c3d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsSuiteBase} import org.apache.spark.sql.execution.datasources.DataSourceUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode @@ -57,7 +57,7 @@ case class SimpleInsert(userSpecifiedSchema: StructType)(@transient val sparkSes } } -class InsertSuite extends DataSourceTest with SharedSparkSession { +class InsertSuite extends DataSourceTest with QueryErrorsSuiteBase with SharedSparkSession { import testImplicits._ protected override lazy val sql = spark.sql _ @@ -1298,8 +1298,11 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { exception = intercept[AnalysisException](sql("insert into t (I) select true from (select 1)")), errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), - parameters = Map("objectName" -> "`I`", "proposal" -> "`i`, `s`")) + errorSubClass = "WITH_SUGGESTION", + sqlState = None, + parameters = Map("objectName" -> "`I`", "proposal" -> "`i`, `s`"), + context = ExpectedContext( + fragment = "insert into t (I)", start = 0, stop = 16)) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala index 593a0339d2ea7..5b0dc4adce9b7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala @@ -21,13 +21,17 @@ import java.time.{Duration, Period} import java.time.temporal.ChronoUnit import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.errors.QueryErrorsSuiteBase import org.apache.spark.sql.execution.datasources.parquet.ParquetTest import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf case class Cases(lower: String, UPPER: String) -class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton { +class HiveParquetSuite extends QueryTest + with ParquetTest + with TestHiveSingleton + with QueryErrorsSuiteBase { test("Case insensitive attribute names") { withParquetTable((1 to 4).map(i => Cases(i.toString, i.toString)), "cases") { @@ -112,24 +116,29 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton test("SPARK-33323: Add query resolved check before convert hive relation") { withTable("t") { + val query = + s""" + |CREATE TABLE t STORED AS PARQUET AS + |SELECT * FROM ( + | SELECT c3 FROM ( + | SELECT c1, c2 from values(1,2) t(c1, c2) + | ) + |) + |""".stripMargin val ex = intercept[AnalysisException] { - sql( - s""" - |CREATE TABLE t STORED AS PARQUET AS - |SELECT * FROM ( - | SELECT c3 FROM ( - | SELECT c1, c2 from values(1,2) t(c1, c2) - | ) - |) - """.stripMargin) + sql(query) } checkError( exception = ex, errorClass = "UNRESOLVED_COLUMN", - errorSubClass = Some("WITH_SUGGESTION"), + errorSubClass = "WITH_SUGGESTION", + sqlState = None, parameters = Map("objectName" -> "`c3`", "proposal" -> ("`__auto_generated_subquery_name`.`c1`, " + - "`__auto_generated_subquery_name`.`c2`"))) + "`__auto_generated_subquery_name`.`c2`")), + context = ExpectedContext( + fragment = query.trim, start = 1, stop = 118) + ) } }