From cae75355e1c055c5f87f93ac54b53cbbb1357043 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 24 Jul 2023 12:31:33 -0400 Subject: [PATCH 01/13] Split AnalysisException --- .../apache/spark/sql/AnalysisException.scala | 11 +-- .../catalyst/ExtendedAnalysisException.scala | 74 +++++++++++++++++++ .../sql/catalyst/analysis/Analyzer.scala | 6 +- .../analysis/ResolveTimeWindows.scala | 4 +- .../UnsupportedOperationChecker.scala | 3 +- .../sql/errors/QueryCompilationErrors.scala | 4 +- .../org/apache/spark/sql/QueryTest.scala | 5 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 8 +- .../apache/spark/sql/SQLQueryTestHelper.scala | 3 +- 9 files changed, 93 insertions(+), 25 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ExtendedAnalysisException.scala 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 ccc3b5f85f70..e1cb02a362be 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 @@ -21,7 +21,6 @@ import scala.collection.JavaConverters._ import org.apache.spark.{QueryContext, SparkThrowable, SparkThrowableHelper} import org.apache.spark.annotation.Stable -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.{Origin, WithOrigin} /** @@ -34,8 +33,6 @@ class AnalysisException protected[sql] ( val message: String, val line: Option[Int] = None, val startPosition: Option[Int] = None, - // Some plans fail to serialize due to bugs in scala collections. - @transient val plan: Option[LogicalPlan] = None, val cause: Option[Throwable] = None, val errorClass: Option[String] = None, val messageParameters: Map[String, String] = Map.empty, @@ -102,12 +99,11 @@ class AnalysisException protected[sql] ( message: String = this.message, line: Option[Int] = this.line, startPosition: Option[Int] = this.startPosition, - plan: Option[LogicalPlan] = this.plan, cause: Option[Throwable] = this.cause, errorClass: Option[String] = this.errorClass, messageParameters: Map[String, String] = this.messageParameters, context: Array[QueryContext] = this.context): AnalysisException = - new AnalysisException(message, line, startPosition, plan, cause, errorClass, + new AnalysisException(message, line, startPosition, cause, errorClass, messageParameters, context) def withPosition(origin: Origin): AnalysisException = { @@ -119,10 +115,7 @@ class AnalysisException protected[sql] ( newException } - override def getMessage: String = { - val planAnnotation = Option(plan).flatten.map(p => s";\n$p").getOrElse("") - getSimpleMessage + planAnnotation - } + override def getMessage: String = getSimpleMessage // Outputs an exception without the logical plan. // For testing only diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ExtendedAnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ExtendedAnalysisException.scala new file mode 100644 index 000000000000..607bac87710a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ExtendedAnalysisException.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst + +import org.apache.spark.QueryContext +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +/** + * Internal [[AnalysisException]] that also captures a [[LogicalPlan]]. + */ +class ExtendedAnalysisException( + message: String, + line: Option[Int] = None, + startPosition: Option[Int] = None, + // Some plans fail to serialize due to bugs in scala collections. + @transient val plan: Option[LogicalPlan] = None, + cause: Option[Throwable] = None, + errorClass: Option[String] = None, + messageParameters: Map[String, String] = Map.empty, + context: Array[QueryContext] = Array.empty) + extends AnalysisException( + message, + line, + startPosition, + cause, + errorClass, + messageParameters, + context) { + + def this(e: AnalysisException, plan: LogicalPlan) { + this( + e.message, + e.line, + e.startPosition, + Option(plan), + e.cause, + e.errorClass, + e.messageParameters, + e.context) + setStackTrace(e.getStackTrace) + } + + override def copy( + message: String, + line: Option[Int], + startPosition: Option[Int], + cause: Option[Throwable], + errorClass: Option[String], + messageParameters: Map[String, String], + context: Array[QueryContext]): ExtendedAnalysisException = { + new ExtendedAnalysisException(message, line, startPosition, plan, cause, errorClass, + messageParameters, context) + } + + override def getMessage: String = { + val planAnnotation = Option(plan).flatten.map(p => s";\n$p").getOrElse("") + getSimpleMessage + planAnnotation + } +} 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 49dea4fa03e5..2750f955e93c 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 @@ -29,7 +29,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.OuterScopes -import org.apache.spark.sql.catalyst.expressions.{Expression, FrameLessOffsetWindowFunction, _} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.objects._ @@ -212,9 +212,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor analyzed } catch { case e: AnalysisException => - val ae = e.copy(plan = Option(analyzed)) - ae.setStackTrace(e.getStackTrace) - throw ae + throw new ExtendedAnalysisException(e, analyzed) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTimeWindows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTimeWindows.scala index 5ce6a531cf09..1ee218f9369c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTimeWindows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTimeWindows.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.ExtendedAnalysisException import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, CaseWhen, Cast, CreateNamedStruct, Expression, GetStructField, IsNotNull, LessThan, Literal, PreciseTimestampConversion, SessionWindow, Subtract, TimeWindow, WindowTime} import org.apache.spark.sql.catalyst.plans.logical.{Expand, Filter, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule @@ -310,7 +310,7 @@ object ResolveWindowTime extends Rule[LogicalPlan] { if (!metadata.contains(TimeWindow.marker) && !metadata.contains(SessionWindow.marker)) { // FIXME: error framework? - throw new AnalysisException( + throw new ExtendedAnalysisException( s"The input is not a correct window column: $windowTime", plan = Some(p)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index 2a09d85d8f26..b86306e6a0c4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.ExtendedAnalysisException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BinaryComparison, CurrentDate, CurrentTimestampLike, Expression, GreaterThan, GreaterThanOrEqual, GroupingSets, LessThan, LessThanOrEqual, LocalTimestamp, MonotonicallyIncreasingID, SessionWindow, WindowExpression} import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ @@ -570,7 +571,7 @@ object UnsupportedOperationChecker extends Logging { } private def throwError(msg: String)(implicit operator: LogicalPlan): Nothing = { - throw new AnalysisException( + throw new ExtendedAnalysisException( msg, operator.origin.line, operator.origin.startPosition, Some(operator)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 8120eb9b57eb..938c3caeadb4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.{SPARK_DOC_ROOT, SparkException, SparkThrowable, SparkThrowableHelper, SparkUnsupportedOperationException} import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{FunctionIdentifier, QualifiedTableName, TableIdentifier} +import org.apache.spark.sql.catalyst.{ExtendedAnalysisException, FunctionIdentifier, QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, FunctionAlreadyExistsException, NamespaceAlreadyExistsException, NoSuchFunctionException, NoSuchNamespaceException, NoSuchPartitionException, NoSuchTableException, ResolvedTable, Star, TableAlreadyExistsException, UnresolvedRegex} import org.apache.spark.sql.catalyst.catalog.{CatalogTable, InvalidUDFClassException} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec @@ -1901,7 +1901,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def streamJoinStreamWithoutEqualityPredicateUnsupportedError(plan: LogicalPlan): Throwable = { val errorClass = "_LEGACY_ERROR_TEMP_1181" - new AnalysisException( + new ExtendedAnalysisException( SparkThrowableHelper.getMessage(errorClass, Map.empty[String, String]), errorClass = Some(errorClass), messageParameters = Map.empty, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 22cc4fd46cbd..c2c333a998b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConverters._ import org.scalatest.Assertions +import org.apache.spark.sql.catalyst.ExtendedAnalysisException import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.SQLExecution @@ -96,7 +97,7 @@ abstract class QueryTest extends PlanTest { private def getResult[T](ds: => Dataset[T]): Array[T] = { val analyzedDS = try ds catch { - case ae: AnalysisException => + case ae: ExtendedAnalysisException => if (ae.plan.isDefined) { fail( s""" @@ -131,7 +132,7 @@ abstract class QueryTest extends PlanTest { */ protected def checkAnswer(df: => DataFrame, expectedAnswer: Seq[Row]): Unit = { val analyzedDF = try df catch { - case ae: AnalysisException => + case ae: ExtendedAnalysisException => if (ae.plan.isDefined) { fail( s""" 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 7ad27f05a58f..4f76dd7a2219 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 @@ -30,6 +30,7 @@ import org.apache.commons.io.FileUtils import org.apache.spark.{AccumulatorSuite, SPARK_DOC_ROOT, SparkException} import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} +import org.apache.spark.sql.catalyst.ExtendedAnalysisException import org.apache.spark.sql.catalyst.expressions.{GenericRow, Hex} import org.apache.spark.sql.catalyst.expressions.Cast._ import org.apache.spark.sql.catalyst.expressions.aggregate.{Complete, Partial} @@ -2634,11 +2635,12 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark assert(!jobStarted.get(), "Command should not trigger a Spark job.") } - test("SPARK-20164: AnalysisException should be tolerant to null query plan") { + test("SPARK-20164: ExtendedAnalysisException should be tolerant to null query plan") { try { - throw new AnalysisException("", None, None, plan = null) + throw new ExtendedAnalysisException("", None, None, plan = null) } catch { - case ae: AnalysisException => assert(ae.plan == null && ae.getMessage == ae.getSimpleMessage) + case ae: ExtendedAnalysisException => + assert(ae.plan == null && ae.getMessage == ae.getSimpleMessage) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala index 5f61e48326aa..d8956961440d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala @@ -140,8 +140,7 @@ trait SQLQueryTestHelper extends Logging { // Do not output the logical plan tree which contains expression IDs. // Also implement a crude way of masking expression IDs in the error message // with a generic pattern "###". - val msg = if (a.plan.nonEmpty) a.getSimpleMessage else a.getMessage - (emptySchema, Seq(a.getClass.getName, msg.replaceAll("#\\d+", "#x"))) + (emptySchema, Seq(a.getClass.getName, a.getSimpleMessage.replaceAll("#\\d+", "#x"))) case s: SparkException if s.getCause != null => // For a runtime exception, it is hard to match because its message contains // information of stage, task ID, etc. From c414f63a9236cc736b6591850d65e84a405229a2 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 24 Jul 2023 12:43:15 -0400 Subject: [PATCH 02/13] Move AnalysisException --- .../apache/spark/sql/AnalysisException.scala | 0 .../spark/sql/catalyst/parser/parsers.scala | 26 +++++++------------ 2 files changed, 10 insertions(+), 16 deletions(-) rename sql/{catalyst => api}/src/main/scala/org/apache/spark/sql/AnalysisException.scala (100%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/api/src/main/scala/org/apache/spark/sql/AnalysisException.scala similarity index 100% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala rename to sql/api/src/main/scala/org/apache/spark/sql/AnalysisException.scala diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala index ac285d54c1fe..39d369fed946 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala @@ -23,9 +23,9 @@ import org.antlr.v4.runtime.atn.PredictionMode import org.antlr.v4.runtime.misc.{Interval, ParseCancellationException} import org.antlr.v4.runtime.tree.TerminalNodeImpl -import org.apache.spark.{QueryContext, SparkException, SparkThrowable, SparkThrowableHelper} +import org.apache.spark.{QueryContext, SparkThrowable, SparkThrowableHelper} import org.apache.spark.internal.Logging -import org.apache.spark.sql.SqlApiConf +import org.apache.spark.sql.{AnalysisException, SqlApiConf} import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin, WithOrigin} import org.apache.spark.sql.catalyst.util.SparkParserUtils import org.apache.spark.sql.errors.QueryParsingErrors @@ -184,15 +184,17 @@ case object ParseErrorListener extends BaseErrorListener { */ class ParseException( val command: Option[String], - val message: String, + message: String, val start: Origin, val stop: Origin, - val errorClass: Option[String] = None, - val messageParameters: Map[String, String] = Map.empty, - val queryContext: Array[QueryContext] = ParseException.getQueryContext()) - extends SparkException( + errorClass: Option[String] = None, + messageParameters: Map[String, String] = Map.empty, + queryContext: Array[QueryContext] = ParseException.getQueryContext()) + extends AnalysisException( message, - cause = null, + start.line, + start.startPosition, + None, errorClass, messageParameters, queryContext) { @@ -222,14 +224,6 @@ class ParseException( Some(errorClass), messageParameters) - // Methods added to retain compatibility with AnalysisException. - @deprecated("Use start.line instead.") - def line: Option[Int] = start.line - @deprecated("Use start.startPosition instead.") - def startPosition: Option[Int] = start.startPosition - @deprecated("ParseException is never caused by another exception.") - def cause: Option[Throwable] = None - override def getMessage: String = { val builder = new StringBuilder builder ++= "\n" ++= message From 392bcbf2d94280419622db0e3a9da937bb744899 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 24 Jul 2023 16:30:00 -0400 Subject: [PATCH 03/13] scala syntax --- .../apache/spark/sql/catalyst/ExtendedAnalysisException.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ExtendedAnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ExtendedAnalysisException.scala index 607bac87710a..2fe1251cff06 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ExtendedAnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ExtendedAnalysisException.scala @@ -42,7 +42,7 @@ class ExtendedAnalysisException( messageParameters, context) { - def this(e: AnalysisException, plan: LogicalPlan) { + def this(e: AnalysisException, plan: LogicalPlan) = { this( e.message, e.line, From c74bc2feb590d19fb90995f936b36cb4fe18c0b7 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 24 Jul 2023 17:32:08 -0400 Subject: [PATCH 04/13] MiMa & Migration Guide --- docs/sql-migration-guide.md | 2 +- project/MimaExcludes.scala | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 42c36ba68189..59b125cbc826 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -28,7 +28,7 @@ license: | - Since Spark 3.5, Spark thrift server will interrupt task when canceling a running statement. To restore the previous behavior, set `spark.sql.thriftServer.interruptOnCancel` to `false`. - Since Spark 3.5, the Avro will throw `AnalysisException` when reading Interval types as Date or Timestamp types, or reading Decimal types with lower precision. To restore the legacy behavior, set `spark.sql.legacy.avro.allowIncompatibleSchema` to `true` - Since Spark 3.5, Row's json and prettyJson methods are moved to `ToJsonUtil`. -- Since Spark 3.5, ParseException is a subclass of SparkException instead of AnalysisException. +- Since Spark 3.5, the `plan` field is moved from `AnalysisException` to `EnhancedAnalysisException`. ## Upgrading from Spark SQL 3.3 to 3.4 diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 729c41755de7..b722ea717070 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -64,7 +64,9 @@ object MimaExcludes { // [SPARK-43952][CORE][CONNECT][SQL] Add SparkContext APIs for query cancellation by tag ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.JobData.this"), // [SPARK-44205][SQL] Extract Catalyst Code from DecimalType - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.types.DecimalType.unapply") + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.types.DecimalType.unapply"), + // [SPARK-44507][SQL][CONNECT] Move AnalysisException to sql/api. + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.AnalysisException") ) // Defulat exclude rules From d2d5e731060300da0114c2476608d314ce719acf Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 24 Jul 2023 18:56:46 -0400 Subject: [PATCH 05/13] More MiMa --- project/MimaExcludes.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index b722ea717070..a110b9042cf9 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -66,10 +66,11 @@ object MimaExcludes { // [SPARK-44205][SQL] Extract Catalyst Code from DecimalType ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.types.DecimalType.unapply"), // [SPARK-44507][SQL][CONNECT] Move AnalysisException to sql/api. - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.AnalysisException") + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.AnalysisException"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.AnalysisException$") ) - // Defulat exclude rules + // Default exclude rules lazy val defaultExcludes = Seq( // Spark Internals ProblemFilters.exclude[Problem]("org.apache.spark.rpc.*"), From feb30cd0677b92220ba4c26e8bda54e1a1eb2c36 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 25 Jul 2023 10:06:45 -0400 Subject: [PATCH 06/13] Make it look like an AnalysisException --- .../sql/catalyst/ExtendedAnalysisException.scala | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ExtendedAnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ExtendedAnalysisException.scala index 2fe1251cff06..2eb7054edceb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ExtendedAnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ExtendedAnalysisException.scala @@ -71,4 +71,17 @@ class ExtendedAnalysisException( val planAnnotation = Option(plan).flatten.map(p => s";\n$p").getOrElse("") getSimpleMessage + planAnnotation } + + override def toString: String = { + val message = getLocalizedMessage + if (message != null) { + ExtendedAnalysisException.name + ": " + message + } else { + ExtendedAnalysisException.name + } + } +} + +object ExtendedAnalysisException { + private val name = classOf[AnalysisException].getName } From 5012175355971809ff9d10a5ed9881785dae54c9 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 25 Jul 2023 10:49:10 -0400 Subject: [PATCH 07/13] Fix tests --- .../resources/sql-tests/results/window.sql.out | 18 +++++++++--------- .../apache/spark/sql/JsonFunctionsSuite.scala | 3 +-- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 7566f9eb20fd..d70f7c974c2a 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -101,7 +101,7 @@ ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", "sqlState" : "42K09", @@ -339,7 +339,7 @@ ORDER BY cate, val_date -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", "sqlState" : "42K09", @@ -381,7 +381,7 @@ ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, v -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_INVALID_BOUND", "sqlState" : "42K09", @@ -406,7 +406,7 @@ RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", "sqlState" : "42K09", @@ -429,7 +429,7 @@ RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_MULTI_ORDER", "sqlState" : "42K09", @@ -453,7 +453,7 @@ RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", "sqlState" : "42K09", @@ -478,7 +478,7 @@ RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY cate, val -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_WRONG_COMPARISON", "sqlState" : "42K09", @@ -1318,7 +1318,7 @@ FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "_LEGACY_ERROR_TEMP_1004", "messageParameters" : { @@ -1335,7 +1335,7 @@ FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "_LEGACY_ERROR_TEMP_1004", "messageParameters" : { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index 2e248a27888c..b21a8344fd00 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -29,7 +29,6 @@ import org.apache.spark.{SparkException, SparkRuntimeException} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Literal, StructsToJson} import org.apache.spark.sql.catalyst.expressions.Cast._ -import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -541,7 +540,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { ) checkError( - exception = intercept[ParseException] { + exception = intercept[AnalysisException] { df3.selectExpr("""from_json(value, 'time InvalidType')""") }, errorClass = "PARSE_SYNTAX_ERROR", From a477dd24f15600ce286960ac656754fc5ea985b8 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 25 Jul 2023 10:49:35 -0400 Subject: [PATCH 08/13] Make it look like an AnalysisException --- .../analyzer-results/ansi/array.sql.out | 10 +- .../analyzer-results/ansi/cast.sql.out | 16 +- .../analyzer-results/ansi/date.sql.out | 18 +- ...double-quoted-identifiers-disabled.sql.out | 10 +- .../double-quoted-identifiers-enabled.sql.out | 22 +- .../analyzer-results/ansi/interval.sql.out | 70 +- .../analyzer-results/ansi/literals.sql.out | 18 +- .../analyzer-results/ansi/map.sql.out | 4 +- .../ansi/string-functions.sql.out | 6 +- .../analyzer-results/ansi/timestamp.sql.out | 8 +- .../ansi/try_arithmetic.sql.out | 2 +- .../sql-tests/analyzer-results/array.sql.out | 10 +- .../analyzer-results/bitwise.sql.out | 4 +- .../analyzer-results/charvarchar.sql.out | 157 +++- .../column-resolution-aggregate.sql.out | 8 +- .../columnresolution-negative.sql.out | 8 +- .../analyzer-results/csv-functions.sql.out | 6 +- .../analyzer-results/cte-legacy.sql.out | 10 +- .../analyzer-results/cte-nested.sql.out | 6 +- .../analyzer-results/cte-nonlegacy.sql.out | 6 +- .../sql-tests/analyzer-results/cte.sql.out | 18 +- .../sql-tests/analyzer-results/date.sql.out | 26 +- .../analyzer-results/datetime-legacy.sql.out | 42 +- .../double-quoted-identifiers.sql.out | 10 +- .../analyzer-results/except-all.sql.out | 4 +- .../analyzer-results/group-analytics.sql.out | 4 +- .../group-by-all-duckdb.sql.out | 4 +- .../group-by-all-mosha.sql.out | 6 +- .../analyzer-results/group-by-all.sql.out | 8 +- .../analyzer-results/group-by-filter.sql.out | 6 +- .../analyzer-results/group-by.sql.out | 36 +- .../analyzer-results/grouping_set.sql.out | 2 +- .../sql-tests/analyzer-results/having.sql.out | 2 +- .../sql-tests/analyzer-results/hll.sql.out | 34 +- .../identifier-clause.sql.out | 2 +- .../analyzer-results/intersect-all.sql.out | 4 +- .../analyzer-results/interval.sql.out | 70 +- .../analyzer-results/join-lateral.sql.out | 24 +- .../analyzer-results/json-functions.sql.out | 10 +- .../sql-tests/analyzer-results/limit.sql.out | 12 +- .../analyzer-results/literals.sql.out | 18 +- .../sql-tests/analyzer-results/map.sql.out | 4 +- .../analyzer-results/mask-functions.sql.out | 12 +- .../named-function-arguments.sql.out | 2 +- .../analyzer-results/natural-join.sql.out | 2 +- .../analyzer-results/null-handling.sql.out | 518 ++++++++++--- .../analyzer-results/order-by-all.sql.out | 2 +- .../analyzer-results/percentiles.sql.out | 24 +- .../sql-tests/analyzer-results/pivot.sql.out | 6 +- .../postgreSQL/aggregates_part1.sql.out | 2 +- .../postgreSQL/aggregates_part3.sql.out | 2 +- .../postgreSQL/create_view.sql.out | 8 +- .../analyzer-results/postgreSQL/join.sql.out | 237 +++--- .../analyzer-results/postgreSQL/limit.sql.out | 4 +- .../postgreSQL/numeric.sql.out | 4 +- .../postgreSQL/select_having.sql.out | 4 +- .../postgreSQL/select_implicit.sql.out | 4 +- .../analyzer-results/postgreSQL/union.sql.out | 2 +- .../postgreSQL/window_part3.sql.out | 126 +++- .../analyzer-results/postgreSQL/with.sql.out | 4 +- .../query_regex_column.sql.out | 16 +- .../sql-tests/analyzer-results/random.sql.out | 4 +- .../analyzer-results/regexp-functions.sql.out | 4 +- .../analyzer-results/show_columns.sql.out | 8 +- .../analyzer-results/string-functions.sql.out | 6 +- .../exists-joins-and-set-ops.sql.out | 16 +- .../subquery/in-subquery/in-basic.sql.out | 2 +- .../in-subquery/in-set-operations.sql.out | 24 +- .../invalid-correlation.sql.out | 8 +- .../subq-input-typecheck.sql.out | 10 +- .../scalar-subquery-predicate.sql.out | 2 +- .../scalar-subquery-set-op.sql.out | 12 +- .../analyzer-results/table-aliases.sql.out | 2 +- .../table-valued-functions.sql.out | 16 +- .../analyzer-results/timestamp.sql.out | 16 +- .../timestampNTZ/timestamp-ansi.sql.out | 8 +- .../timestampNTZ/timestamp.sql.out | 16 +- .../analyzer-results/try_arithmetic.sql.out | 2 +- .../native/booleanEquality.sql.out | 48 +- .../native/caseWhenCoercion.sql.out | 140 ++-- .../native/dateTimeOperations.sql.out | 54 +- .../native/decimalPrecision.sql.out | 704 +++++++++--------- .../typeCoercion/native/division.sql.out | 160 ++-- .../typeCoercion/native/ifCoercion.sql.out | 140 ++-- .../typeCoercion/native/inConversion.sql.out | 280 +++---- .../typeCoercion/native/mapZipWith.sql.out | 4 +- .../typeCoercion/native/mapconcat.sql.out | 10 +- .../native/promoteStrings.sql.out | 94 +-- .../native/stringCastAndExpressions.sql.out | 6 +- .../native/widenSetOperationTypes.sql.out | 140 ++-- .../native/windowFrameCoercion.sql.out | 8 +- .../postgreSQL/udf-aggregates_part1.sql.out | 2 +- .../postgreSQL/udf-aggregates_part3.sql.out | 2 +- .../udf/postgreSQL/udf-join.sql.out | 237 +++--- .../udf/postgreSQL/udf-select_having.sql.out | 4 +- .../postgreSQL/udf-select_implicit.sql.out | 4 +- .../udf/udf-except-all.sql.out | 4 +- .../udf/udf-group-analytics.sql.out | 4 +- .../analyzer-results/udf/udf-group-by.sql.out | 28 +- .../udf/udf-intersect-all.sql.out | 4 +- .../analyzer-results/udf/udf-pivot.sql.out | 6 +- .../analyzer-results/udf/udf-window.sql.out | 12 +- .../sql-tests/analyzer-results/window.sql.out | 18 +- .../sql-tests/results/ansi/array.sql.out | 10 +- .../sql-tests/results/ansi/cast.sql.out | 16 +- .../sql-tests/results/ansi/date.sql.out | 18 +- ...double-quoted-identifiers-disabled.sql.out | 10 +- .../double-quoted-identifiers-enabled.sql.out | 22 +- .../sql-tests/results/ansi/interval.sql.out | 70 +- .../sql-tests/results/ansi/literals.sql.out | 18 +- .../sql-tests/results/ansi/map.sql.out | 4 +- .../results/ansi/string-functions.sql.out | 6 +- .../sql-tests/results/ansi/timestamp.sql.out | 8 +- .../results/ansi/try_arithmetic.sql.out | 2 +- .../resources/sql-tests/results/array.sql.out | 10 +- .../sql-tests/results/bitwise.sql.out | 4 +- .../sql-tests/results/charvarchar.sql.out | 261 ++++--- .../column-resolution-aggregate.sql.out | 8 +- .../results/columnresolution-negative.sql.out | 8 +- .../sql-tests/results/csv-functions.sql.out | 6 +- .../sql-tests/results/cte-legacy.sql.out | 10 +- .../sql-tests/results/cte-nested.sql.out | 6 +- .../sql-tests/results/cte-nonlegacy.sql.out | 6 +- .../resources/sql-tests/results/cte.sql.out | 18 +- .../resources/sql-tests/results/date.sql.out | 26 +- .../sql-tests/results/datetime-legacy.sql.out | 42 +- .../results/double-quoted-identifiers.sql.out | 10 +- .../sql-tests/results/except-all.sql.out | 4 +- .../sql-tests/results/group-analytics.sql.out | 4 +- .../results/group-by-all-duckdb.sql.out | 4 +- .../results/group-by-all-mosha.sql.out | 6 +- .../sql-tests/results/group-by-all.sql.out | 8 +- .../sql-tests/results/group-by-filter.sql.out | 6 +- .../sql-tests/results/group-by.sql.out | 36 +- .../sql-tests/results/grouping_set.sql.out | 2 +- .../sql-tests/results/having.sql.out | 2 +- .../resources/sql-tests/results/hll.sql.out | 32 +- .../results/identifier-clause.sql.out | 2 +- .../sql-tests/results/intersect-all.sql.out | 4 +- .../sql-tests/results/interval.sql.out | 70 +- .../sql-tests/results/join-lateral.sql.out | 24 +- .../sql-tests/results/json-functions.sql.out | 10 +- .../resources/sql-tests/results/limit.sql.out | 12 +- .../sql-tests/results/literals.sql.out | 18 +- .../resources/sql-tests/results/map.sql.out | 4 +- .../sql-tests/results/mask-functions.sql.out | 12 +- .../results/named-function-arguments.sql.out | 2 +- .../sql-tests/results/natural-join.sql.out | 2 +- .../sql-tests/results/null-handling.sql.out | 576 ++++++++++---- .../sql-tests/results/order-by-all.sql.out | 2 +- .../sql-tests/results/percentiles.sql.out | 24 +- .../resources/sql-tests/results/pivot.sql.out | 6 +- .../postgreSQL/aggregates_part1.sql.out | 2 +- .../postgreSQL/aggregates_part3.sql.out | 2 +- .../results/postgreSQL/create_view.sql.out | 8 +- .../sql-tests/results/postgreSQL/join.sql.out | 174 ++++- .../results/postgreSQL/limit.sql.out | 4 +- .../results/postgreSQL/numeric.sql.out | 4 +- .../results/postgreSQL/select_having.sql.out | 4 +- .../postgreSQL/select_implicit.sql.out | 4 +- .../results/postgreSQL/union.sql.out | 2 +- .../results/postgreSQL/window_part3.sql.out | 114 ++- .../sql-tests/results/postgreSQL/with.sql.out | 4 +- .../results/query_regex_column.sql.out | 16 +- .../sql-tests/results/random.sql.out | 4 +- .../results/regexp-functions.sql.out | 4 +- .../sql-tests/results/show_columns.sql.out | 8 +- .../results/string-functions.sql.out | 6 +- .../exists-joins-and-set-ops.sql.out | 16 +- .../subquery/in-subquery/in-basic.sql.out | 2 +- .../in-subquery/in-set-operations.sql.out | 24 +- .../invalid-correlation.sql.out | 8 +- .../subq-input-typecheck.sql.out | 10 +- .../scalar-subquery-predicate.sql.out | 2 +- .../scalar-subquery-set-op.sql.out | 12 +- .../sql-tests/results/table-aliases.sql.out | 2 +- .../results/table-valued-functions.sql.out | 16 +- .../sql-tests/results/timestamp.sql.out | 16 +- .../timestampNTZ/timestamp-ansi.sql.out | 8 +- .../results/timestampNTZ/timestamp.sql.out | 16 +- .../sql-tests/results/try_arithmetic.sql.out | 2 +- .../native/booleanEquality.sql.out | 48 +- .../native/caseWhenCoercion.sql.out | 140 ++-- .../native/dateTimeOperations.sql.out | 54 +- .../native/decimalPrecision.sql.out | 704 +++++++++--------- .../typeCoercion/native/division.sql.out | 160 ++-- .../typeCoercion/native/ifCoercion.sql.out | 140 ++-- .../typeCoercion/native/inConversion.sql.out | 280 +++---- .../typeCoercion/native/mapZipWith.sql.out | 4 +- .../typeCoercion/native/mapconcat.sql.out | 10 +- .../native/promoteStrings.sql.out | 94 +-- .../native/stringCastAndExpressions.sql.out | 6 +- .../native/widenSetOperationTypes.sql.out | 140 ++-- .../native/windowFrameCoercion.sql.out | 8 +- .../postgreSQL/udf-aggregates_part1.sql.out | 2 +- .../postgreSQL/udf-aggregates_part3.sql.out | 2 +- .../results/udf/postgreSQL/udf-join.sql.out | 174 ++++- .../udf/postgreSQL/udf-select_having.sql.out | 4 +- .../postgreSQL/udf-select_implicit.sql.out | 4 +- .../results/udf/udf-except-all.sql.out | 4 +- .../results/udf/udf-group-analytics.sql.out | 4 +- .../results/udf/udf-group-by.sql.out | 28 +- .../results/udf/udf-intersect-all.sql.out | 4 +- .../sql-tests/results/udf/udf-pivot.sql.out | 6 +- .../sql-tests/results/udf/udf-window.sql.out | 12 +- 205 files changed, 4583 insertions(+), 3407 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out index b71d0aad152f..12ad7ce436b8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out @@ -170,7 +170,7 @@ Project [sort_array(boolean_array#x, true) AS sort_array(boolean_array, true)#x, -- !query select sort_array(array('b', 'd'), '1') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -194,7 +194,7 @@ org.apache.spark.sql.AnalysisException -- !query select sort_array(array('b', 'd'), cast(NULL as boolean)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -353,7 +353,7 @@ Project [array_size(cast(null as array)) AS array_size(NULL)#x] -- !query select array_size(map('a', 1, 'b', 2)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -475,7 +475,7 @@ Project [array_insert(array(1, 3, 4), -2, 2) AS array_insert(array(1, 3, 4), -2, -- !query select array_insert(array(1, 2, 3), 3, "4") -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", "sqlState" : "42K09", @@ -534,7 +534,7 @@ Project [array_insert(array(2, 3, cast(null as int), 4), -5, 1) AS array_insert( -- !query select array_compact(id) from values (1) as t(id) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/cast.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/cast.sql.out index 730704b3cbf6..caa324b335c0 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/cast.sql.out @@ -205,7 +205,7 @@ Project [hex(cast(abc as binary)) AS hex(CAST(abc AS BINARY))#x] -- !query SELECT HEX(CAST(CAST(123 AS byte) AS binary)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", @@ -229,7 +229,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT HEX(CAST(CAST(-123 AS byte) AS binary)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", @@ -253,7 +253,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT HEX(CAST(123S AS binary)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", @@ -277,7 +277,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT HEX(CAST(-123S AS binary)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", @@ -301,7 +301,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT HEX(CAST(123 AS binary)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", @@ -325,7 +325,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT HEX(CAST(-123 AS binary)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", @@ -349,7 +349,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT HEX(CAST(123L AS binary)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", @@ -373,7 +373,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT HEX(CAST(-123L AS binary)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/date.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/date.sql.out index 3765d65ec3b7..4373f53dddf8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/date.sql.out @@ -283,7 +283,7 @@ Project [date_add(cast(2011-11-11 as date), 1) AS date_add(2011-11-11, 1)#x] -- !query select date_add('2011-11-11', 1L) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -307,7 +307,7 @@ org.apache.spark.sql.AnalysisException -- !query select date_add('2011-11-11', 1.0) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -331,7 +331,7 @@ org.apache.spark.sql.AnalysisException -- !query select date_add('2011-11-11', 1E1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -421,7 +421,7 @@ Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x] -- !query select date_sub('2011-11-11', 1L) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -445,7 +445,7 @@ org.apache.spark.sql.AnalysisException -- !query select date_sub('2011-11-11', 1.0) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -469,7 +469,7 @@ org.apache.spark.sql.AnalysisException -- !query select date_sub('2011-11-11', 1E1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -574,7 +574,7 @@ Project [date_sub(cast(date_str#x as date), 1) AS date_sub(date_str, 1)#x] -- !query select date '2011-11-11' + 1E1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -658,7 +658,7 @@ select date '2001-09-28' - date_str from date_view -- !query select date'2011-11-11' + '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -682,7 +682,7 @@ org.apache.spark.sql.AnalysisException -- !query select '1' + date'2011-11-11' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-disabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-disabled.sql.out index d7587b0b6588..d73b72eca3e2 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-disabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-disabled.sql.out @@ -114,7 +114,7 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query SELECT 1 FROM `hello` -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -147,7 +147,7 @@ org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException -- !query ALTER TABLE `not_exist` ADD COLUMN not_exist int -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -167,7 +167,7 @@ org.apache.spark.sql.AnalysisException -- !query ALTER TABLE not_exist ADD COLUMN `not_exist` int -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -187,7 +187,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT 1 AS `not_exist` FROM `not_exist` -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -207,7 +207,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT 1 FROM not_exist AS X(`hello`) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out index 2a6bcce99d19..0a009a3a282f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out @@ -2,7 +2,7 @@ -- !query SELECT 1 FROM "not_exist" -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -35,7 +35,7 @@ org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException -- !query ALTER TABLE "not_exist" ADD COLUMN not_exist int -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -55,7 +55,7 @@ org.apache.spark.sql.AnalysisException -- !query ALTER TABLE not_exist ADD COLUMN "not_exist" int -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -75,7 +75,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT 1 AS "not_exist" FROM not_exist -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -95,7 +95,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT 1 FROM not_exist AS X("hello") -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -157,7 +157,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT 1 FROM `hello` -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -190,7 +190,7 @@ org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException -- !query ALTER TABLE `not_exist` ADD COLUMN not_exist int -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -210,7 +210,7 @@ org.apache.spark.sql.AnalysisException -- !query ALTER TABLE not_exist ADD COLUMN `not_exist` int -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -230,7 +230,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT 1 AS `not_exist` FROM `not_exist` -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -250,7 +250,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT 1 FROM not_exist AS X(`hello`) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -312,7 +312,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT "hello" -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/interval.sql.out index 19c77c8de907..1120c40ac15c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/interval.sql.out @@ -159,7 +159,7 @@ Project [(INTERVAL '2' YEAR * cast(a as double)) AS (INTERVAL '2' YEAR * a)#x] -- !query select '2' / interval 2 second -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -181,7 +181,7 @@ org.apache.spark.sql.AnalysisException -- !query select '2' / interval 2 year -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -259,7 +259,7 @@ Project [(INTERVAL '2' YEAR * cast(null as double)) AS (INTERVAL '2' YEAR * NULL -- !query select 2 / interval '2' year -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -281,7 +281,7 @@ org.apache.spark.sql.AnalysisException -- !query select 2 / interval '2' hour -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -303,7 +303,7 @@ org.apache.spark.sql.AnalysisException -- !query select null / interval '2' year -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -325,7 +325,7 @@ org.apache.spark.sql.AnalysisException -- !query select null / interval '2' hour -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1515,7 +1515,7 @@ select '4 day' + interval '10' day, '4 22 day to hour' - interval '10' day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1552,7 +1552,7 @@ Project [(INTERVAL '2' YEAR + cast(null as interval year)) AS (INTERVAL '2' YEAR -- !query select interval '2' year + '3-3' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1574,7 +1574,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval '2' year - '4' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1618,7 +1618,7 @@ CreateViewCommand `interval_view`, select '1' str, false, false, LocalTempView, -- !query select interval '2' year + str from interval_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1640,7 +1640,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval '2' year - str from interval_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1684,7 +1684,7 @@ Project [cast(cast(str#x as timestamp) + INTERVAL '4 22:12' DAY TO MINUTE as str -- !query select interval '2-2' year to month + interval '3' day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1708,7 +1708,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval '3' day + interval '2-2' year to month -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1732,7 +1732,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval '2-2' year to month - interval '3' day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1756,7 +1756,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval '3' day - interval '2-2' year to month -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1778,7 +1778,7 @@ org.apache.spark.sql.AnalysisException -- !query select 1 - interval '2' second -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1802,7 +1802,7 @@ org.apache.spark.sql.AnalysisException -- !query select 1 + interval '2' month -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1824,7 +1824,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval '2' second + 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1848,7 +1848,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval '2' month - 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2694,7 +2694,7 @@ Project [(INTERVAL '-1-1' YEAR TO MONTH = cast(INTERVAL '-13' MONTH as interval -- !query SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2716,7 +2716,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT INTERVAL '1' DAY < '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2738,7 +2738,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT INTERVAL '1' DAY = '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2760,7 +2760,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT INTERVAL '1' DAY > '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2782,7 +2782,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' < INTERVAL '1' DAY -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2804,7 +2804,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' = INTERVAL '1' DAY -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2826,7 +2826,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' > INTERVAL '1' DAY -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2848,7 +2848,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT INTERVAL '1' YEAR < '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2870,7 +2870,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT INTERVAL '1' YEAR = '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2892,7 +2892,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT INTERVAL '1' YEAR > '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2914,7 +2914,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' < INTERVAL '1' YEAR -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2936,7 +2936,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' = INTERVAL '1' YEAR -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2958,7 +2958,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' > INTERVAL '1' YEAR -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2994,7 +2994,7 @@ Project [array(cast(INTERVAL '1' DAY as interval day to minute), cast(INTERVAL ' -- !query SELECT array(INTERVAL 1 MONTH, INTERVAL 20 DAYS) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3030,7 +3030,7 @@ Project [coalesce(cast(INTERVAL '1' DAY as interval day to minute), cast(INTERVA -- !query SELECT coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3094,7 +3094,7 @@ Project [(INTERVAL '1 06' DAY TO HOUR div cast(INTERVAL '-01' HOUR as interval d -- !query SELECT div(INTERVAL '1' MONTH, INTERVAL '-1' DAY) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/literals.sql.out index dd3cf597c95d..53c7327c5871 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/literals.sql.out @@ -474,7 +474,7 @@ Project [3.14 AS 3.14#x, -3.14 AS -3.14#x, 3.14E8 AS 3.14E8#x, 3.14E-8 AS 3.14E- -- !query select +date '1999-01-01' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -498,7 +498,7 @@ org.apache.spark.sql.AnalysisException -- !query select +timestamp '1999-01-01' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -529,7 +529,7 @@ Project [positive(INTERVAL '1' DAY) AS (+ INTERVAL '1' DAY)#x] -- !query select +map(1, 2) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -553,7 +553,7 @@ org.apache.spark.sql.AnalysisException -- !query select +array(1,2) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -577,7 +577,7 @@ org.apache.spark.sql.AnalysisException -- !query select +named_struct('a', 1, 'b', 'spark') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -601,7 +601,7 @@ org.apache.spark.sql.AnalysisException -- !query select +X'1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -625,7 +625,7 @@ org.apache.spark.sql.AnalysisException -- !query select -date '1999-01-01' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -649,7 +649,7 @@ org.apache.spark.sql.AnalysisException -- !query select -timestamp '1999-01-01' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -673,7 +673,7 @@ org.apache.spark.sql.AnalysisException -- !query select -x'2379ACFe' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/map.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/map.sql.out index c55250e4dcb9..177f73608fba 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/map.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/map.sql.out @@ -58,7 +58,7 @@ Project [map_contains_key(map(cast(1.0 as decimal(11,1)), a, cast(2 as decimal(1 -- !query select map_contains_key(map('1', 'a', '2', 'b'), 1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", "sqlState" : "42K09", @@ -82,7 +82,7 @@ org.apache.spark.sql.AnalysisException -- !query select map_contains_key(map(1, 'a', 2, 'b'), '1') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/string-functions.sql.out index 9a66950d3650..f879f2450c99 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/string-functions.sql.out @@ -1266,7 +1266,7 @@ Project [to_binary(cast(null as string), Some(cast(null as string)), false) AS t -- !query select to_binary('abc', 1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", "sqlState" : "42K09", @@ -1290,7 +1290,7 @@ org.apache.spark.sql.AnalysisException -- !query select to_binary('abc', 'invalidFormat') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", "sqlState" : "42K09", @@ -1322,7 +1322,7 @@ CreateViewCommand `fmtTable`, [(fmtField,None)], SELECT * FROM VALUES ('invalidF -- !query SELECT to_binary('abc', fmtField) FROM fmtTable -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/timestamp.sql.out index c4858ac10ae7..d06813dc6751 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/timestamp.sql.out @@ -595,7 +595,7 @@ select timestamp'2011-11-11 11:11:11' - str from ts_view -- !query select timestamp'2011-11-11 11:11:11' + '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", @@ -617,7 +617,7 @@ org.apache.spark.sql.AnalysisException -- !query select '1' + timestamp'2011-11-11 11:11:11' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", @@ -639,7 +639,7 @@ org.apache.spark.sql.AnalysisException -- !query select timestamp'2011-11-11 11:11:11' + null -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -661,7 +661,7 @@ org.apache.spark.sql.AnalysisException -- !query select null + timestamp'2011-11-11 11:11:11' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_arithmetic.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_arithmetic.sql.out index e8506a8589d8..bbc07c22805a 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_arithmetic.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_arithmetic.sql.out @@ -134,7 +134,7 @@ Project [try_add(INTERVAL '02' SECOND, INTERVAL '02' SECOND) AS try_add(INTERVAL -- !query SELECT try_add(interval 2 year, interval 2 second) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out index 91cbd844d998..af06b9959ee5 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out @@ -170,7 +170,7 @@ Project [sort_array(boolean_array#x, true) AS sort_array(boolean_array, true)#x, -- !query select sort_array(array('b', 'd'), '1') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -194,7 +194,7 @@ org.apache.spark.sql.AnalysisException -- !query select sort_array(array('b', 'd'), cast(NULL as boolean)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -353,7 +353,7 @@ Project [array_size(cast(null as array)) AS array_size(NULL)#x] -- !query select array_size(map('a', 1, 'b', 2)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -475,7 +475,7 @@ Project [array_insert(array(1, 3, 4), -2, 2) AS array_insert(array(1, 3, 4), -2, -- !query select array_insert(array(1, 2, 3), 3, "4") -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", "sqlState" : "42K09", @@ -534,7 +534,7 @@ Project [array_insert(array(2, 3, cast(null as int), 4), -5, 1) AS array_insert( -- !query select array_compact(id) from values (1) as t(id) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/bitwise.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/bitwise.sql.out index ca960fc1aca4..96a0000f4f77 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/bitwise.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/bitwise.sql.out @@ -128,7 +128,7 @@ Project [bit_count(-9223372036854775808) AS bit_count(-9223372036854775808)#x] -- !query select bit_count("bit count") -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -152,7 +152,7 @@ org.apache.spark.sql.AnalysisException -- !query select bit_count('a') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out index 6e72fd28686a..c773231d84d9 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out @@ -239,12 +239,13 @@ DescribeTableCommand `spark_catalog`.`default`.`char_tbl1`, true, [col_name#x, d -- !query create table char_part(c1 char(5), c2 char(2), v1 varchar(6), v2 varchar(2)) using parquet partitioned by (v2, c2) -- !query analysis -org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +org.apache.spark.SparkRuntimeException { - "errorClass" : "TABLE_OR_VIEW_ALREADY_EXISTS", - "sqlState" : "42P07", + "errorClass" : "LOCATION_ALREADY_EXISTS", + "sqlState" : "42710", "messageParameters" : { - "relationName" : "`spark_catalog`.`default`.`char_part`" + "identifier" : "`spark_catalog`.`default`.`char_part`", + "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/char_part'" } } @@ -252,71 +253,181 @@ org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException -- !query desc formatted char_part -- !query analysis -DescribeTableCommand `spark_catalog`.`default`.`char_part`, true, [col_name#x, data_type#x, comment#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 24, + "fragment" : "char_part" + } ] +} -- !query alter table char_part add partition (v2='ke', c2='nt') location 'loc1' -- !query analysis -org.apache.spark.sql.catalyst.analysis.PartitionsAlreadyExistException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARTITIONS_ALREADY_EXIST", - "sqlState" : "428FT", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "partitionList" : "PARTITION (`v2` = ke, `c2` = nt)", - "tableName" : "`default`.`char_part`" - } + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "char_part" + } ] } -- !query desc formatted char_part -- !query analysis -DescribeTableCommand `spark_catalog`.`default`.`char_part`, true, [col_name#x, data_type#x, comment#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 24, + "fragment" : "char_part" + } ] +} -- !query alter table char_part partition (v2='ke') rename to partition (v2='nt') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_1232", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "partitionColumnNames" : "v2, c2", - "specKeys" : "v2", - "tableName" : "`spark_catalog`.`default`.`char_part`" - } + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "char_part" + } ] } -- !query desc formatted char_part -- !query analysis -DescribeTableCommand `spark_catalog`.`default`.`char_part`, true, [col_name#x, data_type#x, comment#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 24, + "fragment" : "char_part" + } ] +} -- !query alter table char_part partition (v2='ke', c2='nt') set location 'loc2' -- !query analysis -AlterTableSetLocationCommand `spark_catalog`.`default`.`char_part`, Map(v2 -> ke, c2 -> nt), loc2 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "char_part" + } ] +} -- !query desc formatted char_part -- !query analysis -DescribeTableCommand `spark_catalog`.`default`.`char_part`, true, [col_name#x, data_type#x, comment#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 24, + "fragment" : "char_part" + } ] +} -- !query MSCK REPAIR TABLE char_part -- !query analysis -RepairTableCommand `spark_catalog`.`default`.`char_part`, true, false, MSCK REPAIR TABLE +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 27, + "fragment" : "char_part" + } ] +} -- !query desc formatted char_part -- !query analysis -DescribeTableCommand `spark_catalog`.`default`.`char_part`, true, [col_name#x, data_type#x, comment#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 24, + "fragment" : "char_part" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/column-resolution-aggregate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/column-resolution-aggregate.sql.out index eb30443cbae0..0c213899b64e 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/column-resolution-aggregate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/column-resolution-aggregate.sql.out @@ -18,7 +18,7 @@ CreateViewCommand `v2`, VALUES (1, 1, 1), (2, 2, 1) AS t(x, y, all), false, fals -- !query SELECT max(a) AS b, b FROM v1 GROUP BY k -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -52,7 +52,7 @@ Project [a#x] -- !query SELECT a AS k FROM v1 GROUP BY k -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -66,7 +66,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT x FROM v2 GROUP BY all -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -80,7 +80,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT a AS all, b FROM v1 GROUP BY all -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution-negative.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution-negative.sql.out index c80b9e82ab06..c9306e9d937a 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution-negative.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution-negative.sql.out @@ -255,7 +255,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT db1.t1.i1 FROM t1, mydb2.t1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -289,7 +289,7 @@ SetCatalogAndNamespace -- !query SELECT mydb1.t1 FROM t1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -330,7 +330,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT t1 FROM mydb1.t1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -358,7 +358,7 @@ SetCatalogAndNamespace -- !query SELECT mydb1.t1.i1 FROM t1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/csv-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/csv-functions.sql.out index cfdbc0ce2e9d..cb3feb8d168b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/csv-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/csv-functions.sql.out @@ -36,7 +36,7 @@ org.apache.spark.sql.AnalysisException -- !query select from_csv('1', 'a InvalidType') -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.AnalysisException { "errorClass" : "PARSE_SYNTAX_ERROR", "sqlState" : "42601", @@ -152,7 +152,7 @@ Project [schema_of_csv(1|abc, (delimiter,|)) AS schema_of_csv(1|abc)#x] -- !query select schema_of_csv(null) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL", "sqlState" : "42K09", @@ -181,7 +181,7 @@ CreateViewCommand `csvTable`, [(csvField,None), (a,None)], SELECT * FROM VALUES -- !query SELECT schema_of_csv(csvField) FROM csvTable -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-legacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-legacy.sql.out index 6006c0a47ee9..594a30b054ed 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-legacy.sql.out @@ -52,7 +52,7 @@ SELECT * FROM UNION SELECT * FROM cte -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -327,7 +327,7 @@ WITH ) SELECT * FROM t2 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -381,7 +381,7 @@ SELECT * FROM ( SELECT * FROM cte_inner ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -412,7 +412,7 @@ SELECT * FROM ( SELECT * FROM cte_inner ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -445,7 +445,7 @@ SELECT * FROM ( SELECT * FROM cte_inner ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out index 2902f5c13036..d96965edde13 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out @@ -67,7 +67,7 @@ SELECT * FROM UNION SELECT * FROM cte -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -513,7 +513,7 @@ SELECT * FROM ( SELECT * FROM cte_inner ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -546,7 +546,7 @@ SELECT * FROM ( SELECT * FROM cte_inner ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out index 689bffd58e2d..bd9b443d01d0 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out @@ -67,7 +67,7 @@ SELECT * FROM UNION SELECT * FROM cte -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -589,7 +589,7 @@ SELECT * FROM ( SELECT * FROM cte_inner ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -622,7 +622,7 @@ SELECT * FROM ( SELECT * FROM cte_inner ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out index 1c6e6718fc50..b9a0f776528d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out @@ -20,7 +20,7 @@ CreateViewCommand `t2`, select * from values 0, 1 as t(id), false, false, LocalT -- !query WITH s AS (SELECT 1 FROM s) SELECT * FROM s -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -41,7 +41,7 @@ org.apache.spark.sql.AnalysisException WITH r AS (SELECT (SELECT * FROM r)) SELECT * FROM r -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -79,7 +79,7 @@ WithCTE -- !query WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -251,7 +251,7 @@ org.apache.spark.sql.catalyst.parser.ParseException WITH t AS (SELECT 1 FROM non_existing_table) SELECT 2 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -430,7 +430,7 @@ WithCTE with cte as (select id as id_alias from t) select id from cte -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -453,7 +453,7 @@ with r1 as (select * from r2), r2 as (select 1) select 2 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -521,7 +521,7 @@ SELECT * FROM (WITH q AS (select 1 x) SELECT x+1 AS y FROM q), q -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -728,7 +728,7 @@ WithCTE -- !query with cte AS (SELECT * FROM va) SELECT * FROM cte -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -748,7 +748,7 @@ org.apache.spark.sql.AnalysisException -- !query with cte as (select * from cte) select * from cte -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/date.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/date.sql.out index 42ea145c90fe..227127a353df 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/date.sql.out @@ -283,7 +283,7 @@ Project [date_add(cast(2011-11-11 as date), 1) AS date_add(2011-11-11, 1)#x] -- !query select date_add('2011-11-11', 1L) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -307,7 +307,7 @@ org.apache.spark.sql.AnalysisException -- !query select date_add('2011-11-11', 1.0) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -331,7 +331,7 @@ org.apache.spark.sql.AnalysisException -- !query select date_add('2011-11-11', 1E1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -427,7 +427,7 @@ Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x] -- !query select date_sub('2011-11-11', 1L) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -451,7 +451,7 @@ org.apache.spark.sql.AnalysisException -- !query select date_sub('2011-11-11', 1.0) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -475,7 +475,7 @@ org.apache.spark.sql.AnalysisException -- !query select date_sub('2011-11-11', 1E1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -543,7 +543,7 @@ Project [date_sub(cast(2011-11-11 12:12:12 as date), 1) AS date_sub(TIMESTAMP_NT -- !query select date_add('2011-11-11', int_str) from date_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -567,7 +567,7 @@ org.apache.spark.sql.AnalysisException -- !query select date_sub('2011-11-11', int_str) from date_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -613,7 +613,7 @@ Project [date_sub(cast(date_str#x as date), 1) AS date_sub(date_str, 1)#x] -- !query select date '2011-11-11' + 1E1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -661,7 +661,7 @@ select date '2001-10-01' - date '2001-09-28' -- !query select date '2001-10-01' - '2001-09-28' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -709,7 +709,7 @@ select date_str - date '2001-09-28' from date_view -- !query select date '2001-09-28' - date_str from date_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -733,7 +733,7 @@ org.apache.spark.sql.AnalysisException -- !query select date'2011-11-11' + '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -757,7 +757,7 @@ org.apache.spark.sql.AnalysisException -- !query select '1' + date'2011-11-11' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out index 524a87262b2f..8ae2893a0a8d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out @@ -283,7 +283,7 @@ Project [date_add(cast(2011-11-11 as date), 1) AS date_add(2011-11-11, 1)#x] -- !query select date_add('2011-11-11', 1L) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -307,7 +307,7 @@ org.apache.spark.sql.AnalysisException -- !query select date_add('2011-11-11', 1.0) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -331,7 +331,7 @@ org.apache.spark.sql.AnalysisException -- !query select date_add('2011-11-11', 1E1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -427,7 +427,7 @@ Project [date_sub(cast(2011-11-11 as date), 1) AS date_sub(2011-11-11, 1)#x] -- !query select date_sub('2011-11-11', 1L) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -451,7 +451,7 @@ org.apache.spark.sql.AnalysisException -- !query select date_sub('2011-11-11', 1.0) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -475,7 +475,7 @@ org.apache.spark.sql.AnalysisException -- !query select date_sub('2011-11-11', 1E1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -543,7 +543,7 @@ Project [date_sub(cast(2011-11-11 12:12:12 as date), 1) AS date_sub(TIMESTAMP_NT -- !query select date_add('2011-11-11', int_str) from date_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -567,7 +567,7 @@ org.apache.spark.sql.AnalysisException -- !query select date_sub('2011-11-11', int_str) from date_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -613,7 +613,7 @@ Project [date_sub(cast(date_str#x as date), 1) AS date_sub(date_str, 1)#x] -- !query select date '2011-11-11' + 1E1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -661,7 +661,7 @@ select date '2001-10-01' - date '2001-09-28' -- !query select date '2001-10-01' - '2001-09-28' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -709,7 +709,7 @@ select date_str - date '2001-09-28' from date_view -- !query select date '2001-09-28' - date_str from date_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -733,7 +733,7 @@ org.apache.spark.sql.AnalysisException -- !query select date'2011-11-11' + '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -757,7 +757,7 @@ org.apache.spark.sql.AnalysisException -- !query select '1' + date'2011-11-11' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1582,7 +1582,7 @@ select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' -- !query select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1606,7 +1606,7 @@ org.apache.spark.sql.AnalysisException -- !query select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1650,7 +1650,7 @@ CreateViewCommand `ts_view`, select '2011-11-11 11:11:11' str, false, false, Loc -- !query select str - timestamp'2011-11-11 11:11:11' from ts_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1674,7 +1674,7 @@ org.apache.spark.sql.AnalysisException -- !query select timestamp'2011-11-11 11:11:11' - str from ts_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1698,7 +1698,7 @@ org.apache.spark.sql.AnalysisException -- !query select timestamp'2011-11-11 11:11:11' + '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1720,7 +1720,7 @@ org.apache.spark.sql.AnalysisException -- !query select '1' + timestamp'2011-11-11 11:11:11' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1742,7 +1742,7 @@ org.apache.spark.sql.AnalysisException -- !query select timestamp'2011-11-11 11:11:11' + null -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1764,7 +1764,7 @@ org.apache.spark.sql.AnalysisException -- !query select null + timestamp'2011-11-11 11:11:11' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers.sql.out index d7587b0b6588..d73b72eca3e2 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers.sql.out @@ -114,7 +114,7 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query SELECT 1 FROM `hello` -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -147,7 +147,7 @@ org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException -- !query ALTER TABLE `not_exist` ADD COLUMN not_exist int -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -167,7 +167,7 @@ org.apache.spark.sql.AnalysisException -- !query ALTER TABLE not_exist ADD COLUMN `not_exist` int -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -187,7 +187,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT 1 AS `not_exist` FROM `not_exist` -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -207,7 +207,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT 1 FROM not_exist AS X(`hello`) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/except-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/except-all.sql.out index dedc3f93ba63..fc648785cb53 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/except-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/except-all.sql.out @@ -199,7 +199,7 @@ SELECT * FROM tab1 EXCEPT ALL SELECT array(1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -376,7 +376,7 @@ SELECT k FROM tab3 EXCEPT ALL SELECT k, v FROM tab4 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NUM_COLUMNS_MISMATCH", "sqlState" : "42826", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out index 327caef217a3..592a19f593cf 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out @@ -330,7 +330,7 @@ Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, cast((shi -- !query SELECT course, year, GROUPING(course) FROM courseSales GROUP BY course, year -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", "queryContext" : [ { @@ -346,7 +346,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY course, year -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", "queryContext" : [ { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-duckdb.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-duckdb.sql.out index 53863a812024..59ecafccbeb4 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-duckdb.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-duckdb.sql.out @@ -73,7 +73,7 @@ Sort [((g + i) % 2)#x ASC NULLS FIRST], true -- !query SELECT (g+i)%2 + SUM(i), SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", "sqlState" : "42803", @@ -126,7 +126,7 @@ Sort [c0#x ASC NULLS FIRST], true -- !query SELECT c0 FROM (SELECT 1 c0, 1 c1 UNION ALL SELECT 1, 2) t0 GROUP BY ALL HAVING c1>0 ORDER BY c0 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-mosha.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-mosha.sql.out index a9fc06c1f523..f63f3e8c8452 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-mosha.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-mosha.sql.out @@ -72,7 +72,7 @@ Sort [(i + 1)#x ASC NULLS FIRST, substring(s, 2, 3)#x ASC NULLS FIRST, extract(y -- !query SELECT i + SUM(f) FROM stuff GROUP BY ALL -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", "sqlState" : "42803", @@ -130,7 +130,7 @@ Filter (si#xL < cast(2 as bigint)) -- !query SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING i > 2 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -151,7 +151,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT SUM(i) si FROM stuff GROUP BY ALL ORDER BY i DESC -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all.sql.out index d193e97f79b5..529141e06935 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all.sql.out @@ -224,7 +224,7 @@ Aggregate [count(1) AS count(1)#xL] -- !query select id + count(*) from data group by all -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", "sqlState" : "42803", @@ -241,7 +241,7 @@ org.apache.spark.sql.AnalysisException -- !query select (id + id) / 2 + count(*) * 2 from data group by all -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", "sqlState" : "42803", @@ -315,7 +315,7 @@ Aggregate [country#x, scalar-subquery#x [country#x]], [country#x, scalar-subquer -- !query select (select count(*) from data d1 where d1.country = d2.country) + count(id) from data d2 group by all -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", "sqlState" : "42803", @@ -332,7 +332,7 @@ org.apache.spark.sql.AnalysisException -- !query select non_exist from data group by all -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-filter.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-filter.sql.out index 7b221d8036d6..df2e409fafe7 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-filter.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-filter.sql.out @@ -67,7 +67,7 @@ AS DEPT(dept_id, dept_name, state), false, true, LocalTempView, true -- !query SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -324,7 +324,7 @@ Aggregate [a#x], [a#x, count(b#x) FILTER (WHERE (a#x >= 2)) AS count(b) FILTER ( -- !query SELECT a, COUNT(b) FILTER (WHERE a != 2) FROM testData GROUP BY b -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -754,7 +754,7 @@ Aggregate [(a#x + b#x)], [(a#x + b#x) AS (a + b)#x, count(b#x) FILTER (WHERE (b# -- !query SELECT a + 2, COUNT(b) FILTER (WHERE b IN (1, 2)) FROM testData GROUP BY a + 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by.sql.out index e12af0103d43..67e1ddd32ea8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by.sql.out @@ -15,7 +15,7 @@ AS testData(a, b), false, true, LocalTempView, true -- !query SELECT a, COUNT(b) FROM testData -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -56,7 +56,7 @@ Aggregate [a#x], [a#x, count(b#x) AS count(b)#xL] -- !query SELECT a, COUNT(b) FROM testData GROUP BY b -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -145,7 +145,7 @@ Aggregate [(a#x + b#x)], [(a#x + b#x) AS (a + b)#x, count(b#x) AS count(b)#xL] -- !query SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -230,7 +230,7 @@ Filter (k#x > 1) -- !query SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -251,7 +251,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT COUNT(b) AS k FROM testData GROUP BY k -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "GROUP_BY_AGGREGATE", "sqlState" : "42903", @@ -282,7 +282,7 @@ CreateViewCommand `testDataHasSameNameWithAlias`, SELECT * FROM VALUES -- !query SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -302,7 +302,7 @@ SetCommand (spark.sql.groupByAliases,Some(false)) -- !query SELECT a AS k, COUNT(b) FROM testData GROUP BY k -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -410,7 +410,7 @@ Project [1#x] -- !query SELECT id FROM range(10) HAVING id > 0 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -441,7 +441,7 @@ Filter cast(true as boolean) -- !query SELECT 1 FROM range(10) HAVING MAX(id) > 0 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WHERE_CONDITION", "sqlState" : "42903", @@ -628,7 +628,7 @@ Aggregate [k#x], [k#x, every(v#x) AS every#x] -- !query SELECT every(1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -652,7 +652,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT some(1S) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -676,7 +676,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT any(1L) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -700,7 +700,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT every("true") -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -724,7 +724,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT bool_and(1.0) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -748,7 +748,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT bool_or(1.0D) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -888,7 +888,7 @@ Project [cnt#xL] -- !query SELECT count(*) FROM test_agg WHERE count(*) > 1L -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WHERE_CONDITION", "sqlState" : "42903", @@ -909,7 +909,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WHERE_CONDITION", "sqlState" : "42903", @@ -930,7 +930,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WHERE_CONDITION", "sqlState" : "42903", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out index 8073cc8acc35..bb453923ce95 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out @@ -187,7 +187,7 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query SELECT c1 FROM (values (1,2), (3,2)) t(c1, c2) GROUP BY GROUPING SETS (()) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/having.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/having.sql.out index ec67de27390b..12eb5a34146a 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/having.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/having.sql.out @@ -48,7 +48,7 @@ Project [count(k)#xL] -- !query SELECT count(k) FROM hav GROUP BY v HAVING v = array(1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/hll.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/hll.sql.out index 58391c0054c7..1360458202f4 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/hll.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/hll.sql.out @@ -9,17 +9,35 @@ DropTable true, false -- !query CREATE TABLE t1 USING JSON AS VALUES (0), (1), (2), (2), (2), (3), (4) as tab(col) -- !query analysis -CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`t1`, ErrorIfExists, [col] - +- SubqueryAlias tab - +- LocalRelation [col#x] +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "LOCATION_ALREADY_EXISTS", + "sqlState" : "42710", + "messageParameters" : { + "identifier" : "`spark_catalog`.`default`.`t1`", + "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" + } +} -- !query SELECT hll_sketch_estimate(hll_sketch_agg(col)) AS result FROM t1 -- !query analysis -Aggregate [hll_sketch_estimate(hll_sketch_agg(col#x, 12, 0, 0)) AS result#xL] -+- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[col#x] json +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 64, + "stopIndex" : 65, + "fragment" : "t1" + } ] +} -- !query @@ -80,7 +98,7 @@ Aggregate [hll_sketch_estimate(hll_union_agg(sketch#x, true, 0, 0)) AS hll_sketc SELECT hll_sketch_agg(col) FROM VALUES (ARRAY(1, 2)), (ARRAY(3, 4)) tab(col) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -163,7 +181,7 @@ SELECT hll_union(1, 2) (2, 5), (3, 6) AS tab(col1, col2) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out index 367d492905d5..00e2d8ff8ae7 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out @@ -890,7 +890,7 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query SELECT IDENTIFIER('t').c1 FROM VALUES(1) AS T(c1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/intersect-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/intersect-all.sql.out index 105741b80a20..2535544343d2 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/intersect-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/intersect-all.sql.out @@ -161,7 +161,7 @@ SELECT * FROM tab1 INTERSECT ALL SELECT array(1), 2 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -188,7 +188,7 @@ SELECT k FROM tab1 INTERSECT ALL SELECT k, v FROM tab2 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NUM_COLUMNS_MISMATCH", "sqlState" : "42826", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/interval.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/interval.sql.out index 19abe19e4c65..337edd5980c3 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/interval.sql.out @@ -159,7 +159,7 @@ Project [(INTERVAL '2' YEAR * cast(a as double)) AS (INTERVAL '2' YEAR * a)#x] -- !query select '2' / interval 2 second -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -181,7 +181,7 @@ org.apache.spark.sql.AnalysisException -- !query select '2' / interval 2 year -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -259,7 +259,7 @@ Project [(INTERVAL '2' YEAR * cast(null as double)) AS (INTERVAL '2' YEAR * NULL -- !query select 2 / interval '2' year -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -281,7 +281,7 @@ org.apache.spark.sql.AnalysisException -- !query select 2 / interval '2' hour -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -303,7 +303,7 @@ org.apache.spark.sql.AnalysisException -- !query select null / interval '2' year -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -325,7 +325,7 @@ org.apache.spark.sql.AnalysisException -- !query select null / interval '2' hour -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1515,7 +1515,7 @@ select '4 day' + interval '10' day, '4 22 day to hour' - interval '10' day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1552,7 +1552,7 @@ Project [(INTERVAL '2' YEAR + cast(null as interval year)) AS (INTERVAL '2' YEAR -- !query select interval '2' year + '3-3' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1574,7 +1574,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval '2' year - '4' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1618,7 +1618,7 @@ CreateViewCommand `interval_view`, select '1' str, false, false, LocalTempView, -- !query select interval '2' year + str from interval_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1640,7 +1640,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval '2' year - str from interval_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1684,7 +1684,7 @@ Project [cast(cast(str#x as timestamp) + INTERVAL '4 22:12' DAY TO MINUTE as str -- !query select interval '2-2' year to month + interval '3' day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1708,7 +1708,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval '3' day + interval '2-2' year to month -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1732,7 +1732,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval '2-2' year to month - interval '3' day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1756,7 +1756,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval '3' day - interval '2-2' year to month -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1778,7 +1778,7 @@ org.apache.spark.sql.AnalysisException -- !query select 1 - interval '2' second -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1802,7 +1802,7 @@ org.apache.spark.sql.AnalysisException -- !query select 1 + interval '2' month -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1824,7 +1824,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval '2' second + 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1848,7 +1848,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval '2' month - 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2694,7 +2694,7 @@ Project [(INTERVAL '-1-1' YEAR TO MONTH = cast(INTERVAL '-13' MONTH as interval -- !query SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2716,7 +2716,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT INTERVAL '1' DAY < '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2738,7 +2738,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT INTERVAL '1' DAY = '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2760,7 +2760,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT INTERVAL '1' DAY > '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2782,7 +2782,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' < INTERVAL '1' DAY -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2804,7 +2804,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' = INTERVAL '1' DAY -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2826,7 +2826,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' > INTERVAL '1' DAY -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2848,7 +2848,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT INTERVAL '1' YEAR < '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2870,7 +2870,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT INTERVAL '1' YEAR = '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2892,7 +2892,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT INTERVAL '1' YEAR > '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2914,7 +2914,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' < INTERVAL '1' YEAR -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2936,7 +2936,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' = INTERVAL '1' YEAR -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2958,7 +2958,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' > INTERVAL '1' YEAR -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2994,7 +2994,7 @@ Project [array(cast(INTERVAL '1' DAY as interval day to minute), cast(INTERVAL ' -- !query SELECT array(INTERVAL 1 MONTH, INTERVAL 20 DAYS) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3030,7 +3030,7 @@ Project [coalesce(cast(INTERVAL '1' DAY as interval day to minute), cast(INTERVA -- !query SELECT coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3094,7 +3094,7 @@ Project [(INTERVAL '1 06' DAY TO HOUR div cast(INTERVAL '-01' HOUR as interval d -- !query SELECT div(INTERVAL '1' MONTH, INTERVAL '-1' DAY) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out index 96b6749b8670..5225996c16b7 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out @@ -443,7 +443,7 @@ Project [c1#x, c2#x, c1#x, c2#x, (outer(spark_catalog.default.t1.c2) + outer(spa -- !query SELECT * FROM t1 JOIN LATERAL (SELECT t1.c1 AS a, t2.c1 AS b) s JOIN t2 ON s.b = t2.c1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -475,7 +475,7 @@ SELECT x FROM (SELECT SUM(c1) AS x FROM t1), LATERAL (SELECT x + rand(0) AS y) -- !query SELECT * FROM t1, LATERAL (SELECT c1 + c2 + rand(0) AS c3) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_DETERMINISTIC_LATERAL_SUBQUERIES", "sqlState" : "0A000", @@ -495,7 +495,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT * FROM t1, LATERAL (SELECT rand(0) FROM t2) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_DETERMINISTIC_LATERAL_SUBQUERIES", "sqlState" : "0A000", @@ -515,7 +515,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT * FROM t1 JOIN LATERAL (SELECT * FROM t2) s ON t1.c1 + rand(0) = s.c1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.LATERAL_JOIN_CONDITION_NON_DETERMINISTIC", "sqlState" : "0A000", @@ -655,7 +655,7 @@ Project [c1#x, c2#x, c1#x, m#x, m#x] -- !query SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.c1 + t2.c1)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -675,7 +675,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1), LATERAL (SELECT c2)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -736,7 +736,7 @@ Project [c1#x, c2#x, scalarsubquery(a)#xL] -- !query SELECT * FROM t1, LATERAL (SELECT c1, (SELECT SUM(c2) FROM t2 WHERE c1 = t1.c1)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -1384,7 +1384,7 @@ SELECT * FROM t1 JOIN LATERAL FROM t2 WHERE t2.c1 >= t1.c1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -1915,7 +1915,7 @@ SELECT * FROM t1 JOIN LATERAL WHERE t4.c1 > t1.c2 GROUP BY t4.c2) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", @@ -1941,7 +1941,7 @@ SELECT * FROM t1 JOIN LATERAL SELECT t4.c2 FROM t4) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -2209,7 +2209,7 @@ SELECT * FROM t1 JOIN LATERAL (SELECT sum(c1) FROM ON lhs.c1 <=> rhs.c1 and lhs.c2 <=> rhs.c2 ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -2766,7 +2766,7 @@ Project [col0#x, col1#x] -- !query SELECT t.* FROM t1, LATERAL stack(c1, c2) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/json-functions.sql.out index 79a6bc7de7fd..42b1ab8587bc 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/json-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/json-functions.sql.out @@ -152,7 +152,7 @@ org.apache.spark.sql.AnalysisException -- !query select from_json('{"a":1}', 'a InvalidType') -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.AnalysisException { "errorClass" : "PARSE_SYNTAX_ERROR", "sqlState" : "42601", @@ -428,7 +428,7 @@ Project [schema_of_json({"c1":01, "c2":0.1}, (allowNumericLeadingZeros,true), (p -- !query select schema_of_json(null) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL", "sqlState" : "42K09", @@ -457,7 +457,7 @@ CreateViewCommand `jsonTable`, [(jsonField,None), (a,None)], SELECT * FROM VALUE -- !query SELECT schema_of_json(jsonField) FROM jsonTable -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", @@ -487,7 +487,7 @@ Project [json_array_length(null) AS json_array_length(NULL)#x] -- !query select json_array_length(2) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -620,7 +620,7 @@ Project [json_object_keys(null) AS json_object_keys(NULL)#x] -- !query select json_object_keys(200) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out index f041026f3088..d76a6685c105 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out @@ -52,7 +52,7 @@ GlobalLimit cast(1 as int) -- !query SELECT * FROM testdata LIMIT -1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_NEGATIVE", "messageParameters" : { @@ -73,7 +73,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT * FROM testData TABLESAMPLE (-1 ROWS) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_NEGATIVE", "messageParameters" : { @@ -104,7 +104,7 @@ GlobalLimit cast(1 as int) -- !query SELECT * FROM testdata LIMIT CAST(NULL AS INT) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_NULL", "messageParameters" : { @@ -124,7 +124,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT * FROM testdata LIMIT key > 3 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", "messageParameters" : { @@ -144,7 +144,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT * FROM testdata LIMIT true -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.DATA_TYPE", "messageParameters" : { @@ -158,7 +158,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT * FROM testdata LIMIT 'a' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.DATA_TYPE", "messageParameters" : { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/literals.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/literals.sql.out index dd3cf597c95d..53c7327c5871 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/literals.sql.out @@ -474,7 +474,7 @@ Project [3.14 AS 3.14#x, -3.14 AS -3.14#x, 3.14E8 AS 3.14E8#x, 3.14E-8 AS 3.14E- -- !query select +date '1999-01-01' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -498,7 +498,7 @@ org.apache.spark.sql.AnalysisException -- !query select +timestamp '1999-01-01' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -529,7 +529,7 @@ Project [positive(INTERVAL '1' DAY) AS (+ INTERVAL '1' DAY)#x] -- !query select +map(1, 2) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -553,7 +553,7 @@ org.apache.spark.sql.AnalysisException -- !query select +array(1,2) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -577,7 +577,7 @@ org.apache.spark.sql.AnalysisException -- !query select +named_struct('a', 1, 'b', 'spark') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -601,7 +601,7 @@ org.apache.spark.sql.AnalysisException -- !query select +X'1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -625,7 +625,7 @@ org.apache.spark.sql.AnalysisException -- !query select -date '1999-01-01' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -649,7 +649,7 @@ org.apache.spark.sql.AnalysisException -- !query select -timestamp '1999-01-01' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -673,7 +673,7 @@ org.apache.spark.sql.AnalysisException -- !query select -x'2379ACFe' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/map.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/map.sql.out index 79b12dc68911..cd8f0e043b9a 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/map.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/map.sql.out @@ -58,7 +58,7 @@ Project [map_contains_key(map(cast(1.0 as decimal(11,1)), a, cast(2 as decimal(1 -- !query select map_contains_key(map('1', 'a', '2', 'b'), 1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", "sqlState" : "42K09", @@ -82,7 +82,7 @@ org.apache.spark.sql.AnalysisException -- !query select map_contains_key(map(1, 'a', 2, 'b'), '1') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/mask-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/mask-functions.sql.out index 1abe00ad4709..a339dbf5718c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/mask-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/mask-functions.sql.out @@ -37,7 +37,7 @@ Project [mask(AbCD123-@$#, Q, q, d, o) AS mask(AbCD123-@$#, Q, q, d, o)#x] -- !query SELECT mask('AbCD123-@$#', 'Qa', 'qa', 'da', 'oa') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", "sqlState" : "42K09", @@ -403,7 +403,7 @@ Project [mask(c1#x, null, null, null, null) AS mask(c1, NULL, NULL, NULL, NULL)# -- !query SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 'a') as t(c1, replaceArg) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", @@ -426,7 +426,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 'ABC') as t(c1, replaceArg) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", @@ -449,7 +449,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 123) as t(c1, replaceArg) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -473,7 +473,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT mask('abcd-EFGH-8765-4321', 'A', 'w', '') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", "sqlState" : "42K09", @@ -494,7 +494,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT mask('abcd-EFGH-8765-4321', 'A', 'abc') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out index e01e0ca5ee01..650b61b41924 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/named-function-arguments.sql.out @@ -200,7 +200,7 @@ CreateViewCommand `v`, SELECT id FROM range(0, 8), false, true, LocalTempView, t -- !query SELECT * FROM explode(collection => TABLE(v)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/natural-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/natural-join.sql.out index 987fb3e0a09c..41c8876a7d25 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/natural-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/natural-join.sql.out @@ -488,7 +488,7 @@ Project [v1#x, k#x] -- !query SELECT nt2.k FROM (SELECT * FROM nt1 natural join nt2) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out index 26e9394932a1..9a1e0c52f33e 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out @@ -2,239 +2,565 @@ -- !query create table t1(a int, b int, c int) using parquet -- !query analysis -CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "LOCATION_ALREADY_EXISTS", + "sqlState" : "42710", + "messageParameters" : { + "identifier" : "`spark_catalog`.`default`.`t1`", + "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" + } +} -- !query insert into t1 values(1,0,0) -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] -+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] - +- LocalRelation [col1#x, col2#x, col3#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query insert into t1 values(2,0,1) -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] -+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] - +- LocalRelation [col1#x, col2#x, col3#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query insert into t1 values(3,1,0) -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] -+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] - +- LocalRelation [col1#x, col2#x, col3#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query insert into t1 values(4,1,1) -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] -+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] - +- LocalRelation [col1#x, col2#x, col3#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query insert into t1 values(5,null,0) -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] -+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] - +- LocalRelation [col1#x, col2#x, col3#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query insert into t1 values(6,null,1) -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] -+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] - +- LocalRelation [col1#x, col2#x, col3#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query insert into t1 values(7,null,null) -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] -+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] - +- LocalRelation [col1#x, col2#x, col3#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query select a, b+c from t1 -- !query analysis -Project [a#x, (b#x + c#x) AS (b + c)#x] -+- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 20, + "stopIndex" : 21, + "fragment" : "t1" + } ] +} -- !query select a+10, b*0 from t1 -- !query analysis -Project [(a#x + 10) AS (a + 10)#x, (b#x * 0) AS (b * 0)#x] -+- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 24, + "fragment" : "t1" + } ] +} -- !query select distinct b from t1 -- !query analysis -Distinct -+- Project [b#x] - +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 25, + "fragment" : "t1" + } ] +} -- !query select b from t1 union select b from t1 -- !query analysis -Distinct -+- Union false, false - :- Project [b#x] - : +- SubqueryAlias spark_catalog.default.t1 - : +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet - +- Project [b#x] - +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 16, + "fragment" : "t1" + } ] +} -- !query select a+20, case b when c then 1 else 0 end from t1 -- !query analysis -Project [(a#x + 20) AS (a + 20)#x, CASE WHEN (b#x = c#x) THEN 1 ELSE 0 END AS CASE WHEN (b = c) THEN 1 ELSE 0 END#x] -+- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 52, + "fragment" : "t1" + } ] +} -- !query select a+30, case c when b then 1 else 0 end from t1 -- !query analysis -Project [(a#x + 30) AS (a + 30)#x, CASE WHEN (c#x = b#x) THEN 1 ELSE 0 END AS CASE WHEN (c = b) THEN 1 ELSE 0 END#x] -+- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 52, + "fragment" : "t1" + } ] +} -- !query select a+40, case when b<>0 then 1 else 0 end from t1 -- !query analysis -Project [(a#x + 40) AS (a + 40)#x, CASE WHEN NOT (b#x = 0) THEN 1 ELSE 0 END AS CASE WHEN (NOT (b = 0)) THEN 1 ELSE 0 END#x] -+- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 52, + "stopIndex" : 53, + "fragment" : "t1" + } ] +} -- !query select a+50, case when not b<>0 then 1 else 0 end from t1 -- !query analysis -Project [(a#x + 50) AS (a + 50)#x, CASE WHEN NOT NOT (b#x = 0) THEN 1 ELSE 0 END AS CASE WHEN (NOT (NOT (b = 0))) THEN 1 ELSE 0 END#x] -+- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 56, + "stopIndex" : 57, + "fragment" : "t1" + } ] +} -- !query select a+60, case when b<>0 and c<>0 then 1 else 0 end from t1 -- !query analysis -Project [(a#x + 60) AS (a + 60)#x, CASE WHEN (NOT (b#x = 0) AND NOT (c#x = 0)) THEN 1 ELSE 0 END AS CASE WHEN ((NOT (b = 0)) AND (NOT (c = 0))) THEN 1 ELSE 0 END#x] -+- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 61, + "stopIndex" : 62, + "fragment" : "t1" + } ] +} -- !query select a+70, case when not (b<>0 and c<>0) then 1 else 0 end from t1 -- !query analysis -Project [(a#x + 70) AS (a + 70)#x, CASE WHEN NOT (NOT (b#x = 0) AND NOT (c#x = 0)) THEN 1 ELSE 0 END AS CASE WHEN (NOT ((NOT (b = 0)) AND (NOT (c = 0)))) THEN 1 ELSE 0 END#x] -+- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 67, + "stopIndex" : 68, + "fragment" : "t1" + } ] +} -- !query select a+80, case when b<>0 or c<>0 then 1 else 0 end from t1 -- !query analysis -Project [(a#x + 80) AS (a + 80)#x, CASE WHEN (NOT (b#x = 0) OR NOT (c#x = 0)) THEN 1 ELSE 0 END AS CASE WHEN ((NOT (b = 0)) OR (NOT (c = 0))) THEN 1 ELSE 0 END#x] -+- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 60, + "stopIndex" : 61, + "fragment" : "t1" + } ] +} -- !query select a+90, case when not (b<>0 or c<>0) then 1 else 0 end from t1 -- !query analysis -Project [(a#x + 90) AS (a + 90)#x, CASE WHEN NOT (NOT (b#x = 0) OR NOT (c#x = 0)) THEN 1 ELSE 0 END AS CASE WHEN (NOT ((NOT (b = 0)) OR (NOT (c = 0)))) THEN 1 ELSE 0 END#x] -+- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 66, + "stopIndex" : 67, + "fragment" : "t1" + } ] +} -- !query select count(*), count(b), sum(b), avg(b), min(b), max(b) from t1 -- !query analysis -Aggregate [count(1) AS count(1)#xL, count(b#x) AS count(b)#xL, sum(b#x) AS sum(b)#xL, avg(b#x) AS avg(b)#x, min(b#x) AS min(b)#x, max(b#x) AS max(b)#x] -+- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 64, + "stopIndex" : 65, + "fragment" : "t1" + } ] +} -- !query select a+100 from t1 where b<10 -- !query analysis -Project [(a#x + 100) AS (a + 100)#x] -+- Filter (b#x < 10) - +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 20, + "fragment" : "t1" + } ] +} -- !query select a+110 from t1 where not b>10 -- !query analysis -Project [(a#x + 110) AS (a + 110)#x] -+- Filter NOT (b#x > 10) - +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 20, + "fragment" : "t1" + } ] +} -- !query select a+120 from t1 where b<10 OR c=1 -- !query analysis -Project [(a#x + 120) AS (a + 120)#x] -+- Filter ((b#x < 10) OR (c#x = 1)) - +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 20, + "fragment" : "t1" + } ] +} -- !query select a+130 from t1 where b<10 AND c=1 -- !query analysis -Project [(a#x + 130) AS (a + 130)#x] -+- Filter ((b#x < 10) AND (c#x = 1)) - +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 20, + "fragment" : "t1" + } ] +} -- !query select a+140 from t1 where not (b<10 AND c=1) -- !query analysis -Project [(a#x + 140) AS (a + 140)#x] -+- Filter NOT ((b#x < 10) AND (c#x = 1)) - +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 20, + "fragment" : "t1" + } ] +} -- !query select a+150 from t1 where not (c=1 AND b<10) -- !query analysis -Project [(a#x + 150) AS (a + 150)#x] -+- Filter NOT ((c#x = 1) AND (b#x < 10)) - +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 20, + "fragment" : "t1" + } ] +} -- !query select b, c, equal_null(b, c), equal_null(c, b) from t1 -- !query analysis -Project [b#x, c#x, equal_null(b#x, c#x) AS equal_null(b, c)#x, equal_null(c#x, b#x) AS equal_null(c, b)#x] -+- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 54, + "stopIndex" : 55, + "fragment" : "t1" + } ] +} -- !query drop table t1 -- !query analysis -DropTable false, false -+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1 +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`t1`" + } +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-all.sql.out index 454fff744c40..71f336624593 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-all.sql.out @@ -232,7 +232,7 @@ Sort [all#x ASC NULLS FIRST], true select name, dept, rank() over (partition by dept order by all) as rank from values('Lisa', 'Sales', 10000, 35) as T(name, dept, salary, age) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out index 7a72df5a3e36..fa7529b594ea 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out @@ -204,7 +204,7 @@ SELECT FROM basic_pays ORDER BY salary -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -230,7 +230,7 @@ SELECT FROM basic_pays ORDER BY salary -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -255,7 +255,7 @@ SELECT FROM basic_pays ORDER BY salary -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -281,7 +281,7 @@ SELECT FROM basic_pays ORDER BY salary -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -307,7 +307,7 @@ SELECT FROM basic_pays ORDER BY salary -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -332,7 +332,7 @@ SELECT FROM basic_pays ORDER BY salary -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -414,7 +414,7 @@ FROM basic_pays WINDOW w AS (PARTITION BY department ORDER BY salary) ORDER BY salary -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -441,7 +441,7 @@ FROM basic_pays WINDOW w AS (PARTITION BY department ORDER BY salary) ORDER BY salary -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -467,7 +467,7 @@ FROM basic_pays WINDOW w AS (PARTITION BY department ORDER BY salary) ORDER BY salary -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -494,7 +494,7 @@ FROM basic_pays WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) ORDER BY salary -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -521,7 +521,7 @@ FROM basic_pays WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) ORDER BY salary -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -547,7 +547,7 @@ FROM basic_pays WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) ORDER BY salary -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/pivot.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/pivot.sql.out index d7b77f8ce01f..ead14bdd882d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/pivot.sql.out @@ -346,7 +346,7 @@ PIVOT ( FOR year IN (2012, 2013) ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -396,7 +396,7 @@ PIVOT ( FOR course IN ('dotNET', 'Java') ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NESTED_AGGREGATE_FUNCTION", "sqlState" : "42607", @@ -504,7 +504,7 @@ PIVOT ( FOR year IN (s, 2013) ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part1.sql.out index a17627aa8cb1..beb30cb5e5bf 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part1.sql.out @@ -503,7 +503,7 @@ select (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1))) from tenk1 o -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part3.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part3.sql.out index 2866c7d26c97..2256213b63ff 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part3.sql.out @@ -2,7 +2,7 @@ -- !query select max(min(unique1)) from tenk1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NESTED_AGGREGATE_FUNCTION", "sqlState" : "42607", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out index ac6ee2af69d2..b199cb55f2a4 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out @@ -67,7 +67,7 @@ org.apache.spark.sql.AnalysisException CREATE VIEW key_dependent_view_no_cols AS SELECT FROM view_base_table GROUP BY key HAVING length(data) > 0 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -545,7 +545,7 @@ org.apache.spark.sql.AnalysisException -- !query CREATE VIEW v10_temp AS SELECT * FROM v7_temp -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -565,7 +565,7 @@ org.apache.spark.sql.AnalysisException -- !query CREATE VIEW v11_temp AS SELECT t1.id, t2.a FROM base_table t1, v10_temp t2 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -585,7 +585,7 @@ org.apache.spark.sql.AnalysisException -- !query CREATE VIEW v12_temp AS SELECT true FROM v11_temp -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out index 511c0b397be3..7809cbb57009 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out @@ -655,7 +655,15 @@ Project [ AS xxx#x, i#x, j#x, t#x, k#x] -- !query CREATE TABLE t1 (name STRING, n INTEGER) USING parquet -- !query analysis -CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "LOCATION_ALREADY_EXISTS", + "sqlState" : "42710", + "messageParameters" : { + "identifier" : "`spark_catalog`.`default`.`t1`", + "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" + } +} -- !query @@ -673,9 +681,21 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t3`, false -- !query INSERT INTO t1 VALUES ( 'bb', 11 ) -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [name, n] -+- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] - +- LocalRelation [col1#x, col2#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query @@ -729,17 +749,21 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d -- !query SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name) -- !query analysis -Project [name#x, n#x, n#x, n#x] -+- Project [coalesce(name#x, name#x) AS name#x, n#x, n#x, n#x] - +- Join FullOuter, (name#x = name#x) - :- Project [coalesce(name#x, name#x) AS name#x, n#x, n#x] - : +- Join FullOuter, (name#x = name#x) - : :- SubqueryAlias spark_catalog.default.t1 - : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet - : +- SubqueryAlias spark_catalog.default.t2 - : +- Relation spark_catalog.default.t2[name#x,n#x] parquet - +- SubqueryAlias spark_catalog.default.t3 - +- Relation spark_catalog.default.t3[name#x,n#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 16, + "fragment" : "t1" + } ] +} -- !query @@ -867,23 +891,21 @@ NATURAL INNER JOIN NATURAL INNER JOIN (SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 -- !query analysis -Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] -+- Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] - +- Join Inner, (name#x = name#x) - :- Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x] - : +- Join Inner, (name#x = name#x) - : :- SubqueryAlias s1 - : : +- Project [name#x, n#x AS s1_n#x, 1 AS s1_1#x] - : : +- SubqueryAlias spark_catalog.default.t1 - : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet - : +- SubqueryAlias s2 - : +- Project [name#x, n#x AS s2_n#x, 2 AS s2_2#x] - : +- SubqueryAlias spark_catalog.default.t2 - : +- Relation spark_catalog.default.t2[name#x,n#x] parquet - +- SubqueryAlias s3 - +- Project [name#x, n#x AS s3_n#x, 3 AS s3_2#x] - +- SubqueryAlias spark_catalog.default.t3 - +- Relation spark_catalog.default.t3[name#x,n#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 55, + "stopIndex" : 56, + "fragment" : "t1" + } ] +} -- !query @@ -894,23 +916,21 @@ NATURAL FULL JOIN NATURAL FULL JOIN (SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 -- !query analysis -Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] -+- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] - +- Join FullOuter, (name#x = name#x) - :- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x] - : +- Join FullOuter, (name#x = name#x) - : :- SubqueryAlias s1 - : : +- Project [name#x, n#x AS s1_n#x, 1 AS s1_1#x] - : : +- SubqueryAlias spark_catalog.default.t1 - : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet - : +- SubqueryAlias s2 - : +- Project [name#x, n#x AS s2_n#x, 2 AS s2_2#x] - : +- SubqueryAlias spark_catalog.default.t2 - : +- Relation spark_catalog.default.t2[name#x,n#x] parquet - +- SubqueryAlias s3 - +- Project [name#x, n#x AS s3_n#x, 3 AS s3_2#x] - +- SubqueryAlias spark_catalog.default.t3 - +- Relation spark_catalog.default.t3[name#x,n#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 55, + "stopIndex" : 56, + "fragment" : "t1" + } ] +} -- !query @@ -923,25 +943,21 @@ NATURAL FULL JOIN (SELECT name, n as s3_n FROM t3) as s3 ) ss2 -- !query analysis -Project [name#x, s1_n#x, s2_n#x, s3_n#x] -+- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s2_n#x, s3_n#x] - +- Join FullOuter, (name#x = name#x) - :- SubqueryAlias s1 - : +- Project [name#x, n#x AS s1_n#x] - : +- SubqueryAlias spark_catalog.default.t1 - : +- Relation spark_catalog.default.t1[name#x,n#x] parquet - +- SubqueryAlias ss2 - +- Project [name#x, s2_n#x, s3_n#x] - +- Project [coalesce(name#x, name#x) AS name#x, s2_n#x, s3_n#x] - +- Join FullOuter, (name#x = name#x) - :- SubqueryAlias s2 - : +- Project [name#x, n#x AS s2_n#x] - : +- SubqueryAlias spark_catalog.default.t2 - : +- Relation spark_catalog.default.t2[name#x,n#x] parquet - +- SubqueryAlias s3 - +- Project [name#x, n#x AS s3_n#x] - +- SubqueryAlias spark_catalog.default.t3 - +- Relation spark_catalog.default.t3[name#x,n#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 45, + "fragment" : "t1" + } ] +} -- !query @@ -954,25 +970,21 @@ NATURAL FULL JOIN (SELECT name, n as s3_n FROM t3) as s3 ) ss2 -- !query analysis -Project [name#x, s1_n#x, s2_n#x, s2_2#x, s3_n#x] -+- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s2_n#x, s2_2#x, s3_n#x] - +- Join FullOuter, (name#x = name#x) - :- SubqueryAlias s1 - : +- Project [name#x, n#x AS s1_n#x] - : +- SubqueryAlias spark_catalog.default.t1 - : +- Relation spark_catalog.default.t1[name#x,n#x] parquet - +- SubqueryAlias ss2 - +- Project [name#x, s2_n#x, s2_2#x, s3_n#x] - +- Project [coalesce(name#x, name#x) AS name#x, s2_n#x, s2_2#x, s3_n#x] - +- Join FullOuter, (name#x = name#x) - :- SubqueryAlias s2 - : +- Project [name#x, n#x AS s2_n#x, 2 AS s2_2#x] - : +- SubqueryAlias spark_catalog.default.t2 - : +- Relation spark_catalog.default.t2[name#x,n#x] parquet - +- SubqueryAlias s3 - +- Project [name#x, n#x AS s3_n#x] - +- SubqueryAlias spark_catalog.default.t3 - +- Relation spark_catalog.default.t3[name#x,n#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 45, + "fragment" : "t1" + } ] +} -- !query @@ -982,16 +994,21 @@ FULL JOIN (SELECT name, 2 as s2_n FROM t2) as s2 ON (s1_n = s2_n) -- !query analysis -Project [name#x, s1_n#x, name#x, s2_n#x] -+- Join FullOuter, (s1_n#x = s2_n#x) - :- SubqueryAlias s1 - : +- Project [name#x, n#x AS s1_n#x] - : +- SubqueryAlias spark_catalog.default.t1 - : +- Relation spark_catalog.default.t1[name#x,n#x] parquet - +- SubqueryAlias s2 - +- Project [name#x, 2 AS s2_n#x] - +- SubqueryAlias spark_catalog.default.t2 - +- Relation spark_catalog.default.t2[name#x,n#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 46, + "stopIndex" : 47, + "fragment" : "t1" + } ] +} -- !query @@ -1586,8 +1603,14 @@ Aggregate [count(1) AS count(1)#xL] -- !query DROP TABLE t1 -- !query analysis -DropTable false, false -+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1 +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`t1`" + } +} -- !query @@ -3008,7 +3031,7 @@ org.apache.spark.sql.AnalysisException select * from int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = y.f1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3065,7 +3088,7 @@ Project [q1#xL, q2#xL, f1#x, ff#x] select t1.uunique1 from tenk1 t1 join tenk2 t2 on t1.two = t2.two -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3087,7 +3110,7 @@ org.apache.spark.sql.AnalysisException select t2.uunique1 from tenk1 t1 join tenk2 t2 on t1.two = t2.two -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3109,7 +3132,7 @@ org.apache.spark.sql.AnalysisException select uunique1 from tenk1 t1 join tenk2 t2 on t1.two = t2.two -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3433,7 +3456,7 @@ Project [q1#xL, q2#xL, bq1#xL, cq1#xL, least(outer(a.q1), q1, q1)#xL] -- !query select f1,g from int4_tbl a, (select f1 as g) ss -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -3453,7 +3476,7 @@ org.apache.spark.sql.AnalysisException -- !query select f1,g from int4_tbl a, (select a.f1 as g) ss -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -3473,7 +3496,7 @@ org.apache.spark.sql.AnalysisException -- !query select f1,g from int4_tbl a cross join (select f1 as g) ss -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -3493,7 +3516,7 @@ org.apache.spark.sql.AnalysisException -- !query select f1,g from int4_tbl a cross join (select a.f1 as g) ss -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/limit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/limit.sql.out index 495f425dd5ee..2105423dcdd7 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/limit.sql.out @@ -139,7 +139,7 @@ CreateViewCommand `INT8_TBL`, SELECT * FROM -- !query select * from int8_tbl limit (case when random() < 0.5 then bigint(null) end) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", "messageParameters" : { @@ -159,7 +159,7 @@ org.apache.spark.sql.AnalysisException -- !query select * from int8_tbl offset (case when random() < 0.5 then bigint(null) end) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", "messageParameters" : { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out index 536631179d83..418e3b2626f8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out @@ -4519,7 +4519,7 @@ Project [ AS to_number_1#x, to_number(-34,338,492, 99G999G999) AS to_number(-34, -- !query SELECT '' AS to_number_2, to_number('-34,338,492.654,878', '99G999G999D999G999') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_FORMAT.THOUSANDS_SEPS_MUST_BEFORE_DEC", "sqlState" : "42601", @@ -4574,7 +4574,7 @@ Project [ AS to_number_14#x, to_number(34,50, 999,99) AS to_number(34,50, 999,99 -- !query SELECT '' AS to_number_15, to_number('123,000','999G') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_FORMAT.CONT_THOUSANDS_SEPS", "sqlState" : "42601", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_having.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_having.sql.out index 15df35070603..215e93abb5a5 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_having.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_having.sql.out @@ -142,7 +142,7 @@ Filter (min(a)#x < max(a)#x) -- !query SELECT a FROM test_having HAVING min(a) < max(a) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -159,7 +159,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT 1 AS one FROM test_having HAVING a > 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_implicit.sql.out index 687eeee8f6ac..d2e8345fc942 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_implicit.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_implicit.sql.out @@ -107,7 +107,7 @@ Project [count(1)#xL] -- !query SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -319,7 +319,7 @@ Project [count(c)#xL] -- !query SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/union.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/union.sql.out index 56975bd020e1..343767d49b0d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/union.sql.out @@ -996,7 +996,7 @@ Sort [q2#xL ASC NULLS FIRST, q1#xL ASC NULLS FIRST], true -- !query SELECT q1 FROM int8_tbl EXCEPT SELECT q2 FROM int8_tbl ORDER BY q2 LIMIT 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out index 020cdbf375c5..f77be5a62b67 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out @@ -215,15 +215,35 @@ GlobalLimit 0 -- !query create table t1 (f1 int, f2 int) using parquet -- !query analysis -CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "LOCATION_ALREADY_EXISTS", + "sqlState" : "42710", + "messageParameters" : { + "identifier" : "`spark_catalog`.`default`.`t1`", + "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" + } +} -- !query insert into t1 values (1,1),(1,2),(2,2) -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [f1, f2] -+- Project [cast(col1#x as int) AS f1#x, cast(col2#x as int) AS f2#x] - +- LocalRelation [col1#x, col2#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query @@ -231,19 +251,19 @@ select f1, sum(f1) over (partition by f1 range between 1 preceding and 1 following) from t1 where f1 = f2 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", - "sqlState" : "42K09", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "sqlExpr" : "\"(PARTITION BY f1 RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)\"" + "relationName" : "`t1`" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 25, - "stopIndex" : 108, - "fragment" : "(partition by f1\n range between 1 preceding and 1 following)" + "startIndex" : 115, + "stopIndex" : 116, + "fragment" : "t1" } ] } @@ -253,13 +273,21 @@ select f1, sum(f1) over (partition by f1 order by f2 range between 1 preceding and 1 following) from t1 where f1 = f2 -- !query analysis -Project [f1#x, sum(f1) OVER (PARTITION BY f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL] -+- Project [f1#x, f2#x, sum(f1) OVER (PARTITION BY f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL, sum(f1) OVER (PARTITION BY f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL] - +- Window [sum(f1#x) windowspecdefinition(f1#x, f2#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -1, 1)) AS sum(f1) OVER (PARTITION BY f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [f1#x], [f2#x ASC NULLS FIRST] - +- Project [f1#x, f2#x] - +- Filter (f1#x = f2#x) - +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[f1#x,f2#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 102, + "stopIndex" : 103, + "fragment" : "t1" + } ] +} -- !query @@ -267,13 +295,21 @@ select f1, sum(f1) over (partition by f1, f1 order by f2 range between 2 preceding and 1 preceding) from t1 where f1 = f2 -- !query analysis -Project [f1#x, sum(f1) OVER (PARTITION BY f1, f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL] -+- Project [f1#x, f2#x, sum(f1) OVER (PARTITION BY f1, f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL, sum(f1) OVER (PARTITION BY f1, f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL] - +- Window [sum(f1#x) windowspecdefinition(f1#x, f1#x, f2#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -2, -1)) AS sum(f1) OVER (PARTITION BY f1, f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL], [f1#x, f1#x], [f2#x ASC NULLS FIRST] - +- Project [f1#x, f2#x] - +- Filter (f1#x = f2#x) - +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[f1#x,f2#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 106, + "stopIndex" : 107, + "fragment" : "t1" + } ] +} -- !query @@ -281,13 +317,21 @@ select f1, sum(f1) over (partition by f1, f2 order by f2 range between 1 following and 2 following) from t1 where f1 = f2 -- !query analysis -Project [f1#x, sum(f1) OVER (PARTITION BY f1, f2 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 FOLLOWING AND 2 FOLLOWING)#xL] -+- Project [f1#x, f2#x, sum(f1) OVER (PARTITION BY f1, f2 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 FOLLOWING AND 2 FOLLOWING)#xL, sum(f1) OVER (PARTITION BY f1, f2 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 FOLLOWING AND 2 FOLLOWING)#xL] - +- Window [sum(f1#x) windowspecdefinition(f1#x, f2#x, f2#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, 1, 2)) AS sum(f1) OVER (PARTITION BY f1, f2 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 FOLLOWING AND 2 FOLLOWING)#xL], [f1#x, f2#x], [f2#x ASC NULLS FIRST] - +- Project [f1#x, f2#x] - +- Filter (f1#x = f2#x) - +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[f1#x,f2#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 106, + "stopIndex" : 107, + "fragment" : "t1" + } ] +} -- !query @@ -315,7 +359,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT * FROM empsalary INNER JOIN tenk1 ON row_number() OVER (ORDER BY salary) < 10 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_EXPR_FOR_OPERATOR", "messageParameters" : { @@ -334,7 +378,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT rank() OVER (ORDER BY 1), count(*) FROM empsalary GROUP BY 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_EXPR_FOR_OPERATOR", "messageParameters" : { @@ -426,7 +470,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT ntile(0) OVER (ORDER BY ten), ten, four FROM tenk1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", "sqlState" : "42K09", @@ -449,7 +493,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT nth_value(four, 0) OVER (ORDER BY ten), ten, four FROM tenk1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", "sqlState" : "42K09", @@ -486,5 +530,11 @@ DropTable false, false -- !query DROP TABLE t1 -- !query analysis -DropTable false, false -+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1 +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`t1`" + } +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out index e53480e96bed..c978c583152c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out @@ -240,7 +240,7 @@ WITH outermost(x) AS ( ) SELECT * FROM outermost ORDER BY 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -424,7 +424,7 @@ WithCTE -- !query WITH test AS (SELECT 42) INSERT INTO test VALUES (1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/query_regex_column.sql.out index 4e75fb74de30..c8e9dcde4478 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/query_regex_column.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/query_regex_column.sql.out @@ -34,7 +34,7 @@ AS testData2(A, B, c, d), false, true, LocalTempView, true -- !query SELECT `(a)?+.+` FROM testData2 WHERE a = 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -55,7 +55,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -76,7 +76,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT `(a|b)` FROM testData2 WHERE a = 2 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -97,7 +97,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -118,7 +118,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT SUM(`(a|b)?+.+`) FROM testData2 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -139,7 +139,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT SUM(`(a)`) FROM testData2 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -456,7 +456,7 @@ Aggregate [a#x], [sum(a#x) AS sum(a)#xL] -- !query SELECT SUM(a) FROM testdata3 GROUP BY `(a)` -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -477,7 +477,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT SUM(a) FROM testdata3 GROUP BY `(a)?+.+` -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out index de5dc246c0e8..e0c051e0a934 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out @@ -26,7 +26,7 @@ SELECT rand(cast(NULL AS int)) -- !query SELECT rand(1.0) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -74,7 +74,7 @@ SELECT randn(cast(NULL AS long)) -- !query SELECT rand('1') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/regexp-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/regexp-functions.sql.out index 1d30315e3f91..cd6b95e75f1d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/regexp-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/regexp-functions.sql.out @@ -234,7 +234,7 @@ Project [regexp_replace(healthy, wealthy, and wise, \w+thy, something, 1) AS reg -- !query SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', -2) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", "sqlState" : "42K09", @@ -257,7 +257,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 0) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/show_columns.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/show_columns.sql.out index 3c790c07891d..55c744a8c072 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/show_columns.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/show_columns.sql.out @@ -66,7 +66,7 @@ ShowColumnsCommand showdb, `spark_catalog`.`showdb`.`showcolumn2`, [col_name#x] -- !query SHOW COLUMNS IN badtable FROM showdb -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -111,7 +111,7 @@ ShowColumnsCommand `showcolumn3`, [col_name#x] -- !query SHOW COLUMNS IN showdb.showcolumn3 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -131,7 +131,7 @@ org.apache.spark.sql.AnalysisException -- !query SHOW COLUMNS IN showcolumn3 FROM showdb -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -151,7 +151,7 @@ org.apache.spark.sql.AnalysisException -- !query SHOW COLUMNS IN showcolumn4 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/string-functions.sql.out index 9a66950d3650..f879f2450c99 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/string-functions.sql.out @@ -1266,7 +1266,7 @@ Project [to_binary(cast(null as string), Some(cast(null as string)), false) AS t -- !query select to_binary('abc', 1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", "sqlState" : "42K09", @@ -1290,7 +1290,7 @@ org.apache.spark.sql.AnalysisException -- !query select to_binary('abc', 'invalidFormat') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", "sqlState" : "42K09", @@ -1322,7 +1322,7 @@ CreateViewCommand `fmtTable`, [(fmtField,None)], SELECT * FROM VALUES ('invalidF -- !query SELECT to_binary('abc', fmtField) FROM fmtTable -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out index 8ea5bf7baf1c..7d098783eba9 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out @@ -685,7 +685,7 @@ WHERE EXISTS (SELECT * FROM dept WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -713,7 +713,7 @@ WHERE NOT EXISTS (SELECT * FROM dept WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -741,7 +741,7 @@ WHERE EXISTS (SELECT * FROM dept WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -769,7 +769,7 @@ WHERE EXISTS (SELECT * FROM dept WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -797,7 +797,7 @@ WHERE EXISTS (SELECT * FROM dept WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -825,7 +825,7 @@ WHERE EXISTS (SELECT * FROM dept WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -853,7 +853,7 @@ WHERE NOT EXISTS (SELECT * FROM dept WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -881,7 +881,7 @@ WHERE NOT EXISTS (SELECT * FROM dept WHERE dept_id = emp.dept_id and state = "TX") -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-basic.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-basic.sql.out index 8541900a0eae..bfb89e61a668 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-basic.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-basic.sql.out @@ -50,7 +50,7 @@ Project [1 AS 1#x] -- !query select 1 from tab_a where (a1, b1) not in (select (a2, b2) from tab_b) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_LENGTH_MISMATCH", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-set-operations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-set-operations.sql.out index 937173cac743..f088251de621 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-set-operations.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-set-operations.sql.out @@ -1313,7 +1313,7 @@ WHERE t1a IN (SELECT t2a SELECT t3a FROM t3) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -1340,7 +1340,7 @@ WHERE t1a IN (SELECT t2a SELECT t3a FROM t3) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -1367,7 +1367,7 @@ WHERE t1a IN (SELECT t2a SELECT t3a FROM t3) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -1394,7 +1394,7 @@ WHERE t1a IN (SELECT t2a SELECT t3a FROM t3) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -1421,7 +1421,7 @@ WHERE t1a IN (SELECT t2a SELECT t3a FROM t3) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -1448,7 +1448,7 @@ WHERE t1a IN (SELECT t2a SELECT t3a FROM t3) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -1475,7 +1475,7 @@ WHERE t1a NOT IN (SELECT t2a SELECT t3a FROM t3) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -1502,7 +1502,7 @@ WHERE t1a NOT IN (SELECT t2a SELECT t3a FROM t3) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -1529,7 +1529,7 @@ WHERE t1a NOT IN (SELECT t2a SELECT t3a FROM t3) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -1556,7 +1556,7 @@ WHERE t1a NOT IN (SELECT t2a SELECT t3a FROM t3) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -1583,7 +1583,7 @@ WHERE t1a NOT IN (SELECT t2a SELECT t3a FROM t3) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -1610,7 +1610,7 @@ WHERE t1a NOT IN (SELECT t2a SELECT t3a FROM t3) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out index 2992bc6c9a11..e17633460e0c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/invalid-correlation.sql.out @@ -49,7 +49,7 @@ AND t2b = (SELECT max(avg) ) ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -74,7 +74,7 @@ WHERE t1a IN (SELECT min(t2a) GROUP BY t3b HAVING t3b > t2b )) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", "messageParameters" : { @@ -153,7 +153,7 @@ WHERE t1a IN (SELECT t2a WHERE EXISTS (SELECT min(t2a) FROM t3)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", @@ -189,7 +189,7 @@ FROM t1 JOIN t1_copy ON EXISTS (SELECT 1 FROM t2 WHERE t2a > t1a) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/subq-input-typecheck.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/subq-input-typecheck.sql.out index 864c589184e5..e550e52bf64b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/subq-input-typecheck.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/negative-cases/subq-input-typecheck.sql.out @@ -73,7 +73,7 @@ SELECT ) FROM t1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_SUBQUERY_EXPRESSION.SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", "sqlState" : "42823", @@ -99,7 +99,7 @@ SELECT ) FROM t1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_SUBQUERY_EXPRESSION.SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", "sqlState" : "42823", @@ -123,7 +123,7 @@ t1a IN (SELECT t2a, t2b FROM t2 WHERE t1a = t2a) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_LENGTH_MISMATCH", "sqlState" : "42K09", @@ -151,7 +151,7 @@ WHERE FROM t2 WHERE t1a = t2a) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_LENGTH_MISMATCH", "sqlState" : "42K09", @@ -180,7 +180,7 @@ WHERE t5c FROM t5) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_DATA_TYPE_MISMATCH", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out index 311f3803902e..a55b1e717be1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out @@ -1189,7 +1189,7 @@ SELECT * FROM t0 WHERE t0a < FROM t2) ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out index 0f53d05ac3e7..790d9da94e14 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out @@ -299,7 +299,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", @@ -603,7 +603,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", @@ -899,7 +899,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", @@ -1195,7 +1195,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", @@ -1491,7 +1491,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", @@ -1787,7 +1787,7 @@ SELECT t0a, (SELECT sum(d) FROM ) FROM t0 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/table-aliases.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/table-aliases.sql.out index f15382d66a56..0c8d0b8f2693 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/table-aliases.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/table-aliases.sql.out @@ -95,7 +95,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT a AS col1, b AS col2 FROM testData AS t(c, d) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/table-valued-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/table-valued-functions.sql.out index 6c29a0ec1db7..2b292eb6895b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/table-valued-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/table-valued-functions.sql.out @@ -214,7 +214,7 @@ select * from explode(array(rand(0))) -- !query select * from explode(null) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -238,7 +238,7 @@ org.apache.spark.sql.AnalysisException -- !query select * from explode(null) t(c1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -262,7 +262,7 @@ org.apache.spark.sql.AnalysisException -- !query select * from explode(1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -428,7 +428,7 @@ Project [col1#x, col2#x] -- !query select * from inline(null) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -541,7 +541,7 @@ Project [pos#x, k#x, v#x] -- !query select * from posexplode(1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -735,7 +735,7 @@ org.apache.spark.sql.AnalysisException -- !query select * from json_tuple('{"a": 1}', 1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE", "sqlState" : "42K09", @@ -756,7 +756,7 @@ org.apache.spark.sql.AnalysisException -- !query select * from json_tuple('{"a": 1}', null) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE", "sqlState" : "42K09", @@ -891,7 +891,7 @@ org.apache.spark.sql.AnalysisException -- !query select * from stack(2, 1, '1.1', 'a', 2, 2.2, 'b') -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.STACK_COLUMN_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp.sql.out index 856c317d1374..12fa38ac3c3a 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp.sql.out @@ -551,7 +551,7 @@ select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' -- !query select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -575,7 +575,7 @@ org.apache.spark.sql.AnalysisException -- !query select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -619,7 +619,7 @@ CreateViewCommand `ts_view`, select '2011-11-11 11:11:11' str, false, false, Loc -- !query select str - timestamp'2011-11-11 11:11:11' from ts_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -643,7 +643,7 @@ org.apache.spark.sql.AnalysisException -- !query select timestamp'2011-11-11 11:11:11' - str from ts_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -667,7 +667,7 @@ org.apache.spark.sql.AnalysisException -- !query select timestamp'2011-11-11 11:11:11' + '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -689,7 +689,7 @@ org.apache.spark.sql.AnalysisException -- !query select '1' + timestamp'2011-11-11 11:11:11' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -711,7 +711,7 @@ org.apache.spark.sql.AnalysisException -- !query select timestamp'2011-11-11 11:11:11' + null -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -733,7 +733,7 @@ org.apache.spark.sql.AnalysisException -- !query select null + timestamp'2011-11-11 11:11:11' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp-ansi.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp-ansi.sql.out index 5c42401acc35..bbdbd903d823 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp-ansi.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp-ansi.sql.out @@ -609,7 +609,7 @@ Project [(2011-11-11 11:11:11 - cast(str#x as timestamp_ntz)) AS (TIMESTAMP_NTZ -- !query select timestamp'2011-11-11 11:11:11' + '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", @@ -631,7 +631,7 @@ org.apache.spark.sql.AnalysisException -- !query select '1' + timestamp'2011-11-11 11:11:11' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", @@ -653,7 +653,7 @@ org.apache.spark.sql.AnalysisException -- !query select timestamp'2011-11-11 11:11:11' + null -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -675,7 +675,7 @@ org.apache.spark.sql.AnalysisException -- !query select null + timestamp'2011-11-11 11:11:11' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp.sql.out index 18d05088b566..720ab45acd71 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestampNTZ/timestamp.sql.out @@ -553,7 +553,7 @@ select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' -- !query select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -577,7 +577,7 @@ org.apache.spark.sql.AnalysisException -- !query select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -621,7 +621,7 @@ CreateViewCommand `ts_view`, select '2011-11-11 11:11:11' str, false, false, Loc -- !query select str - timestamp'2011-11-11 11:11:11' from ts_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -645,7 +645,7 @@ org.apache.spark.sql.AnalysisException -- !query select timestamp'2011-11-11 11:11:11' - str from ts_view -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -669,7 +669,7 @@ org.apache.spark.sql.AnalysisException -- !query select timestamp'2011-11-11 11:11:11' + '1' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -691,7 +691,7 @@ org.apache.spark.sql.AnalysisException -- !query select '1' + timestamp'2011-11-11 11:11:11' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -713,7 +713,7 @@ org.apache.spark.sql.AnalysisException -- !query select timestamp'2011-11-11 11:11:11' + null -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -735,7 +735,7 @@ org.apache.spark.sql.AnalysisException -- !query select null + timestamp'2011-11-11 11:11:11' -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/try_arithmetic.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/try_arithmetic.sql.out index e8506a8589d8..bbc07c22805a 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/try_arithmetic.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/try_arithmetic.sql.out @@ -134,7 +134,7 @@ Project [try_add(INTERVAL '02' SECOND, INTERVAL '02' SECOND) AS try_add(INTERVAL -- !query SELECT try_add(interval 2 year, interval 2 second) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/booleanEquality.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/booleanEquality.sql.out index 2f2b27e427d0..c1aa8f3c7921 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/booleanEquality.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/booleanEquality.sql.out @@ -98,7 +98,7 @@ Project [(true = cast(cast(1 as string) as boolean)) AS (true = CAST(1 AS STRING -- !query SELECT true = cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -131,7 +131,7 @@ Project [(true = cast(1 as boolean)) AS (true = CAST(1 AS BOOLEAN))#x] -- !query SELECT true = cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -153,7 +153,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT true = cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -263,7 +263,7 @@ Project [(true <=> cast(cast(1 as string) as boolean)) AS (true <=> CAST(1 AS ST -- !query SELECT true <=> cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -296,7 +296,7 @@ Project [(true <=> cast(1 as boolean)) AS (true <=> CAST(1 AS BOOLEAN))#x] -- !query SELECT true <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -318,7 +318,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT true <=> cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -428,7 +428,7 @@ Project [(cast(cast(1 as string) as boolean) = true) AS (CAST(1 AS STRING) = tru -- !query SELECT cast('1' as binary) = true FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -461,7 +461,7 @@ Project [(cast(1 as boolean) = true) AS (CAST(1 AS BOOLEAN) = true)#x] -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) = true FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -483,7 +483,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) = true FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -593,7 +593,7 @@ Project [(cast(cast(1 as string) as boolean) <=> true) AS (CAST(1 AS STRING) <=> -- !query SELECT cast('1' as binary) <=> true FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -626,7 +626,7 @@ Project [(cast(1 as boolean) <=> true) AS (CAST(1 AS BOOLEAN) <=> true)#x] -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> true FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -648,7 +648,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) <=> true FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -758,7 +758,7 @@ Project [(false = cast(cast(0 as string) as boolean)) AS (false = CAST(0 AS STRI -- !query SELECT false = cast('0' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -791,7 +791,7 @@ Project [(false = cast(0 as boolean)) AS (false = CAST(0 AS BOOLEAN))#x] -- !query SELECT false = cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -813,7 +813,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT false = cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -923,7 +923,7 @@ Project [(false <=> cast(cast(0 as string) as boolean)) AS (false <=> CAST(0 AS -- !query SELECT false <=> cast('0' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -956,7 +956,7 @@ Project [(false <=> cast(0 as boolean)) AS (false <=> CAST(0 AS BOOLEAN))#x] -- !query SELECT false <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -978,7 +978,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT false <=> cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1088,7 +1088,7 @@ Project [(cast(cast(0 as string) as boolean) = false) AS (CAST(0 AS STRING) = fa -- !query SELECT cast('0' as binary) = false FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1121,7 +1121,7 @@ Project [(cast(0 as boolean) = false) AS (CAST(0 AS BOOLEAN) = false)#x] -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) = false FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1143,7 +1143,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) = false FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1253,7 +1253,7 @@ Project [(cast(cast(0 as string) as boolean) <=> false) AS (CAST(0 AS STRING) <= -- !query SELECT cast('0' as binary) <=> false FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1286,7 +1286,7 @@ Project [(cast(0 as boolean) <=> false) AS (CAST(0 AS BOOLEAN) <=> false)#x] -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> false FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1308,7 +1308,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) <=> false FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/caseWhenCoercion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/caseWhenCoercion.sql.out index 6742bb25fd9f..7662eac61e54 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/caseWhenCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/caseWhenCoercion.sql.out @@ -98,7 +98,7 @@ Project [CASE WHEN true THEN cast(cast(1 as tinyint) as string) ELSE cast(2 as s -- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2' as binary) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -120,7 +120,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as boolean) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -142,7 +142,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -164,7 +164,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00' as date) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -274,7 +274,7 @@ Project [CASE WHEN true THEN cast(cast(1 as smallint) as string) ELSE cast(2 as -- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2' as binary) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -296,7 +296,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as boolean) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -318,7 +318,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -340,7 +340,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00' as date) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -450,7 +450,7 @@ Project [CASE WHEN true THEN cast(cast(1 as int) as string) ELSE cast(2 as strin -- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2' as binary) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -472,7 +472,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as boolean) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -494,7 +494,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -516,7 +516,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00' as date) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -626,7 +626,7 @@ Project [CASE WHEN true THEN cast(cast(1 as bigint) as string) ELSE cast(2 as st -- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2' as binary) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -648,7 +648,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as boolean) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -670,7 +670,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -692,7 +692,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00' as date) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -802,7 +802,7 @@ Project [CASE WHEN true THEN cast(cast(1 as float) as string) ELSE cast(2 as str -- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2' as binary) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -824,7 +824,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as boolean) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -846,7 +846,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -868,7 +868,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00' as date) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -978,7 +978,7 @@ Project [CASE WHEN true THEN cast(cast(1 as double) as string) ELSE cast(2 as st -- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2' as binary) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1000,7 +1000,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as boolean) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1022,7 +1022,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1044,7 +1044,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00' as date) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1154,7 +1154,7 @@ Project [CASE WHEN true THEN cast(cast(1 as decimal(10,0)) as string) ELSE cast( -- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2' as binary) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1176,7 +1176,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as boolean) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1198,7 +1198,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1220,7 +1220,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30:00' as date) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1330,7 +1330,7 @@ Project [CASE WHEN true THEN cast(1 as string) ELSE cast(2 as string) END AS CAS -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2' as binary) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1352,7 +1352,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as boolean) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1396,7 +1396,7 @@ Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2017-12-11 09:30:0 -- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as tinyint) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1418,7 +1418,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as smallint) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1440,7 +1440,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as int) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1462,7 +1462,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as bigint) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1484,7 +1484,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as float) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1506,7 +1506,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as double) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1528,7 +1528,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as decimal(10, 0)) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1550,7 +1550,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as string) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1583,7 +1583,7 @@ Project [CASE WHEN true THEN cast(1 as binary) ELSE cast(2 as binary) END AS CAS -- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as boolean) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1605,7 +1605,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1627,7 +1627,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00' as date) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1649,7 +1649,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as tinyint) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1671,7 +1671,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as smallint) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1693,7 +1693,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as int) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1715,7 +1715,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as bigint) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1737,7 +1737,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as float) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1759,7 +1759,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as double) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1781,7 +1781,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as decimal(10, 0)) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1803,7 +1803,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as string) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1825,7 +1825,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2' as binary) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1858,7 +1858,7 @@ Project [CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as boolean) END AS C -- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1880,7 +1880,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00' as date) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1902,7 +1902,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as tinyint) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1924,7 +1924,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as smallint) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1946,7 +1946,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as int) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1968,7 +1968,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as bigint) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1990,7 +1990,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as float) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2012,7 +2012,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as double) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2034,7 +2034,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as decimal(10, 0)) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2067,7 +2067,7 @@ Project [CASE WHEN true THEN cast(cast(2017-12-12 09:30:00.0 as timestamp) as st -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast('2' as binary) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2089,7 +2089,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as boolean) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2133,7 +2133,7 @@ Project [CASE WHEN true THEN cast(2017-12-12 09:30:00.0 as timestamp) ELSE cast( -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as tinyint) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2155,7 +2155,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as smallint) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2177,7 +2177,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as int) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2199,7 +2199,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as bigint) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2221,7 +2221,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as float) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2243,7 +2243,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as double) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2265,7 +2265,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as decimal(10, 0)) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2298,7 +2298,7 @@ Project [CASE WHEN true THEN cast(cast(2017-12-12 09:30:00 as date) as string) E -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2' as binary) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2320,7 +2320,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as boolean) END FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/dateTimeOperations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/dateTimeOperations.sql.out index a4d0d0fd3702..4e5884d1219b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/dateTimeOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/dateTimeOperations.sql.out @@ -10,7 +10,7 @@ CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, true -- !query select cast(1 as tinyint) + interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -34,7 +34,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(1 as smallint) + interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -58,7 +58,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(1 as int) + interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -82,7 +82,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(1 as bigint) + interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -106,7 +106,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(1 as float) + interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -130,7 +130,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(1 as double) + interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -154,7 +154,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(1 as decimal(10, 0)) + interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -192,7 +192,7 @@ Project [cast(cast(cast(2017-12-11 09:30:00 as string) as timestamp) + INTERVAL -- !query select cast('1' as binary) + interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -216,7 +216,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(1 as boolean) + interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -254,7 +254,7 @@ Project [date_add(cast(2017-12-11 09:30:00 as date), extractansiintervaldays(INT -- !query select interval 2 day + cast(1 as tinyint) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -278,7 +278,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval 2 day + cast(1 as smallint) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -302,7 +302,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval 2 day + cast(1 as int) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -326,7 +326,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval 2 day + cast(1 as bigint) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -350,7 +350,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval 2 day + cast(1 as float) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -374,7 +374,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval 2 day + cast(1 as double) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -398,7 +398,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval 2 day + cast(1 as decimal(10, 0)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -436,7 +436,7 @@ Project [cast(cast(cast(2017-12-11 09:30:00 as string) as timestamp) + INTERVAL -- !query select interval 2 day + cast('1' as binary) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -460,7 +460,7 @@ org.apache.spark.sql.AnalysisException -- !query select interval 2 day + cast(1 as boolean) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -498,7 +498,7 @@ Project [date_add(cast(2017-12-11 09:30:00 as date), extractansiintervaldays(INT -- !query select cast(1 as tinyint) - interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -522,7 +522,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(1 as smallint) - interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -546,7 +546,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(1 as int) - interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -570,7 +570,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(1 as bigint) - interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -594,7 +594,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(1 as float) - interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -618,7 +618,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(1 as double) - interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -642,7 +642,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(1 as decimal(10, 0)) - interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -680,7 +680,7 @@ Project [cast(cast(2017-12-11 09:30:00 as string) - INTERVAL '2' DAY as string) -- !query select cast('1' as binary) - interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -704,7 +704,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(1 as boolean) - interval 2 day -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/decimalPrecision.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/decimalPrecision.sql.out index c69150bda486..0ab4530c6b96 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/decimalPrecision.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/decimalPrecision.sql.out @@ -318,7 +318,7 @@ Project [(cast(1 as decimal(10,0)) + cast(1 as decimal(20,0))) AS (CAST(1 AS DEC -- !query SELECT cast('1' as binary) + cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -340,7 +340,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) + cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -362,7 +362,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) + cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -384,7 +384,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) + cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -406,7 +406,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -428,7 +428,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -450,7 +450,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -472,7 +472,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -494,7 +494,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -518,7 +518,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -542,7 +542,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -566,7 +566,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -942,7 +942,7 @@ Project [(cast(cast(1 as decimal(20,0)) as double) + cast(cast(1 as string) as d -- !query SELECT cast(1 as decimal(3, 0)) + cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -964,7 +964,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) + cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -986,7 +986,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) + cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1008,7 +1008,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) + cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1030,7 +1030,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) + cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1052,7 +1052,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) + cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1074,7 +1074,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) + cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1096,7 +1096,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) + cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1118,7 +1118,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1140,7 +1140,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1162,7 +1162,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1184,7 +1184,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1206,7 +1206,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1230,7 +1230,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1254,7 +1254,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1278,7 +1278,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1610,7 +1610,7 @@ Project [(cast(1 as decimal(10,0)) - cast(1 as decimal(20,0))) AS (CAST(1 AS DEC -- !query SELECT cast('1' as binary) - cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1632,7 +1632,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) - cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1654,7 +1654,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) - cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1676,7 +1676,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) - cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1698,7 +1698,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1722,7 +1722,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1746,7 +1746,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1770,7 +1770,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1794,7 +1794,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1818,7 +1818,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1842,7 +1842,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1866,7 +1866,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2242,7 +2242,7 @@ Project [(cast(cast(1 as decimal(20,0)) as double) - cast(cast(1 as string) as d -- !query SELECT cast(1 as decimal(3, 0)) - cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2264,7 +2264,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) - cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2286,7 +2286,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) - cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2308,7 +2308,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) - cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2330,7 +2330,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) - cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2352,7 +2352,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) - cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2374,7 +2374,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) - cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2396,7 +2396,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) - cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2418,7 +2418,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2442,7 +2442,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2466,7 +2466,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2490,7 +2490,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2514,7 +2514,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2538,7 +2538,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2562,7 +2562,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2586,7 +2586,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2918,7 +2918,7 @@ Project [(cast(1 as decimal(10,0)) * cast(1 as decimal(20,0))) AS (CAST(1 AS DEC -- !query SELECT cast('1' as binary) * cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2940,7 +2940,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) * cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2962,7 +2962,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) * cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2984,7 +2984,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) * cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3006,7 +3006,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3028,7 +3028,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3050,7 +3050,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3072,7 +3072,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3094,7 +3094,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3116,7 +3116,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3138,7 +3138,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3160,7 +3160,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3534,7 +3534,7 @@ Project [(cast(cast(1 as decimal(20,0)) as double) * cast(cast(1 as string) as d -- !query SELECT cast(1 as decimal(3, 0)) * cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3556,7 +3556,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) * cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3578,7 +3578,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) * cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3600,7 +3600,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) * cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3622,7 +3622,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) * cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3644,7 +3644,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) * cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3666,7 +3666,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) * cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3688,7 +3688,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) * cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3710,7 +3710,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3732,7 +3732,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3754,7 +3754,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3776,7 +3776,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3798,7 +3798,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3820,7 +3820,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3842,7 +3842,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3864,7 +3864,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4194,7 +4194,7 @@ Project [(cast(1 as decimal(10,0)) / cast(1 as decimal(20,0))) AS (CAST(1 AS DEC -- !query SELECT cast('1' as binary) / cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4216,7 +4216,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) / cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4238,7 +4238,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4260,7 +4260,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) / cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4282,7 +4282,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4304,7 +4304,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4326,7 +4326,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4348,7 +4348,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4370,7 +4370,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4392,7 +4392,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4414,7 +4414,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4436,7 +4436,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4810,7 +4810,7 @@ Project [(cast(cast(1 as decimal(20,0)) as double) / cast(cast(1 as string) as d -- !query SELECT cast(1 as decimal(3, 0)) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4832,7 +4832,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4854,7 +4854,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4876,7 +4876,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4898,7 +4898,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4920,7 +4920,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4942,7 +4942,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4964,7 +4964,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4986,7 +4986,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5008,7 +5008,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5030,7 +5030,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5052,7 +5052,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5074,7 +5074,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5096,7 +5096,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5118,7 +5118,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5140,7 +5140,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5470,7 +5470,7 @@ Project [(cast(1 as decimal(10,0)) % cast(1 as decimal(20,0))) AS (CAST(1 AS DEC -- !query SELECT cast('1' as binary) % cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5492,7 +5492,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) % cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5514,7 +5514,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) % cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5536,7 +5536,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) % cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5558,7 +5558,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5580,7 +5580,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5602,7 +5602,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5624,7 +5624,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5646,7 +5646,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5668,7 +5668,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5690,7 +5690,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5712,7 +5712,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6086,7 +6086,7 @@ Project [(cast(cast(1 as decimal(20,0)) as double) % cast(cast(1 as string) as d -- !query SELECT cast(1 as decimal(3, 0)) % cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6108,7 +6108,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) % cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6130,7 +6130,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) % cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6152,7 +6152,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) % cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6174,7 +6174,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) % cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6196,7 +6196,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) % cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6218,7 +6218,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) % cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6240,7 +6240,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) % cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6262,7 +6262,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6284,7 +6284,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6306,7 +6306,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6328,7 +6328,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6350,7 +6350,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6372,7 +6372,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6394,7 +6394,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6416,7 +6416,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6746,7 +6746,7 @@ Project [pmod(cast(1 as decimal(10,0)), cast(1 as decimal(20,0))) AS pmod(CAST(1 -- !query SELECT pmod(cast('1' as binary), cast(1 as decimal(3, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6768,7 +6768,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast('1' as binary), cast(1 as decimal(5, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6790,7 +6790,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast('1' as binary), cast(1 as decimal(10, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6812,7 +6812,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast('1' as binary), cast(1 as decimal(20, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6834,7 +6834,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(3, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6856,7 +6856,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(5, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6878,7 +6878,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(10, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6900,7 +6900,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(20, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6922,7 +6922,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(3, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6944,7 +6944,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(5, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6966,7 +6966,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(10, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6988,7 +6988,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(20, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7362,7 +7362,7 @@ Project [pmod(cast(cast(1 as decimal(20,0)) as double), cast(cast(1 as string) a -- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7384,7 +7384,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7406,7 +7406,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as decimal(10, 0)), cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7428,7 +7428,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as decimal(20, 0)), cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7450,7 +7450,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7472,7 +7472,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7494,7 +7494,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7516,7 +7516,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7538,7 +7538,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7560,7 +7560,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7582,7 +7582,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7604,7 +7604,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7626,7 +7626,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7648,7 +7648,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7670,7 +7670,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7692,7 +7692,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8022,7 +8022,7 @@ Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) = cast(1 as decimal(20 -- !query SELECT cast('1' as binary) = cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8044,7 +8044,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) = cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8066,7 +8066,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) = cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8088,7 +8088,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) = cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8110,7 +8110,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8132,7 +8132,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8154,7 +8154,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8176,7 +8176,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8198,7 +8198,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8220,7 +8220,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8242,7 +8242,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8264,7 +8264,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8638,7 +8638,7 @@ Project [(cast(cast(1 as decimal(20,0)) as double) = cast(cast(1 as string) as d -- !query SELECT cast(1 as decimal(3, 0)) = cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8660,7 +8660,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) = cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8682,7 +8682,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) = cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8704,7 +8704,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) = cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8770,7 +8770,7 @@ Project [(cast(1 as decimal(20,0)) = cast(cast(1 as boolean) as decimal(20,0))) -- !query SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8792,7 +8792,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8814,7 +8814,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8836,7 +8836,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8858,7 +8858,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8880,7 +8880,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8902,7 +8902,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8924,7 +8924,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9254,7 +9254,7 @@ Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) <=> cast(1 as decimal( -- !query SELECT cast('1' as binary) <=> cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9276,7 +9276,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) <=> cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9298,7 +9298,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) <=> cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9320,7 +9320,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) <=> cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9342,7 +9342,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9364,7 +9364,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9386,7 +9386,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9408,7 +9408,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9430,7 +9430,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9452,7 +9452,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9474,7 +9474,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9496,7 +9496,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9870,7 +9870,7 @@ Project [(cast(cast(1 as decimal(20,0)) as double) <=> cast(cast(1 as string) as -- !query SELECT cast(1 as decimal(3, 0)) <=> cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9892,7 +9892,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) <=> cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9914,7 +9914,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) <=> cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9936,7 +9936,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) <=> cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10002,7 +10002,7 @@ Project [(cast(1 as decimal(20,0)) <=> cast(cast(1 as boolean) as decimal(20,0)) -- !query SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10024,7 +10024,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10046,7 +10046,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10068,7 +10068,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10090,7 +10090,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10112,7 +10112,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10134,7 +10134,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10156,7 +10156,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10486,7 +10486,7 @@ Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) < cast(1 as decimal(20 -- !query SELECT cast('1' as binary) < cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10508,7 +10508,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) < cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10530,7 +10530,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) < cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10552,7 +10552,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) < cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10574,7 +10574,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10596,7 +10596,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10618,7 +10618,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10640,7 +10640,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10662,7 +10662,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10684,7 +10684,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10706,7 +10706,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10728,7 +10728,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11102,7 +11102,7 @@ Project [(cast(cast(1 as decimal(20,0)) as double) < cast(cast(1 as string) as d -- !query SELECT cast(1 as decimal(3, 0)) < cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11124,7 +11124,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) < cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11146,7 +11146,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) < cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11168,7 +11168,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) < cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11190,7 +11190,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) < cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11212,7 +11212,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) < cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11234,7 +11234,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) < cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11256,7 +11256,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) < cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11278,7 +11278,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11300,7 +11300,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11322,7 +11322,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11344,7 +11344,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11366,7 +11366,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11388,7 +11388,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11410,7 +11410,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11432,7 +11432,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11762,7 +11762,7 @@ Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) <= cast(1 as decimal(2 -- !query SELECT cast('1' as binary) <= cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11784,7 +11784,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) <= cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11806,7 +11806,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) <= cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11828,7 +11828,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) <= cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11850,7 +11850,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11872,7 +11872,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11894,7 +11894,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11916,7 +11916,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11938,7 +11938,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11960,7 +11960,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11982,7 +11982,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12004,7 +12004,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12378,7 +12378,7 @@ Project [(cast(cast(1 as decimal(20,0)) as double) <= cast(cast(1 as string) as -- !query SELECT cast(1 as decimal(3, 0)) <= cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12400,7 +12400,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) <= cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12422,7 +12422,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) <= cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12444,7 +12444,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) <= cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12466,7 +12466,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) <= cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12488,7 +12488,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) <= cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12510,7 +12510,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) <= cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12532,7 +12532,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) <= cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12554,7 +12554,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12576,7 +12576,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12598,7 +12598,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12620,7 +12620,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12642,7 +12642,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12664,7 +12664,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12686,7 +12686,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12708,7 +12708,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13038,7 +13038,7 @@ Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) > cast(1 as decimal(20 -- !query SELECT cast('1' as binary) > cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13060,7 +13060,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) > cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13082,7 +13082,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) > cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13104,7 +13104,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) > cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13126,7 +13126,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13148,7 +13148,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13170,7 +13170,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13192,7 +13192,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13214,7 +13214,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13236,7 +13236,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13258,7 +13258,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13280,7 +13280,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13654,7 +13654,7 @@ Project [(cast(cast(1 as decimal(20,0)) as double) > cast(cast(1 as string) as d -- !query SELECT cast(1 as decimal(3, 0)) > cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13676,7 +13676,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) > cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13698,7 +13698,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) > cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13720,7 +13720,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) > cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13742,7 +13742,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) > cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13764,7 +13764,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) > cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13786,7 +13786,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) > cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13808,7 +13808,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) > cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13830,7 +13830,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13852,7 +13852,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13874,7 +13874,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13896,7 +13896,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13918,7 +13918,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13940,7 +13940,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13962,7 +13962,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13984,7 +13984,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14314,7 +14314,7 @@ Project [(cast(cast(1 as decimal(10,0)) as decimal(20,0)) >= cast(1 as decimal(2 -- !query SELECT cast('1' as binary) >= cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14336,7 +14336,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) >= cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14358,7 +14358,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) >= cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14380,7 +14380,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) >= cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14402,7 +14402,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14424,7 +14424,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14446,7 +14446,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14468,7 +14468,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14490,7 +14490,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14512,7 +14512,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14534,7 +14534,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14556,7 +14556,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14930,7 +14930,7 @@ Project [(cast(cast(1 as decimal(20,0)) as double) >= cast(cast(1 as string) as -- !query SELECT cast(1 as decimal(3, 0)) >= cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14952,7 +14952,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) >= cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14974,7 +14974,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) >= cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14996,7 +14996,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) >= cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15018,7 +15018,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) >= cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15040,7 +15040,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) >= cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15062,7 +15062,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) >= cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15084,7 +15084,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) >= cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15106,7 +15106,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15128,7 +15128,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15150,7 +15150,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15172,7 +15172,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15194,7 +15194,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15216,7 +15216,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15238,7 +15238,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15260,7 +15260,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15590,7 +15590,7 @@ Project [NOT (cast(cast(1 as decimal(10,0)) as decimal(20,0)) = cast(1 as decima -- !query SELECT cast('1' as binary) <> cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15612,7 +15612,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) <> cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15634,7 +15634,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) <> cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15656,7 +15656,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) <> cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15678,7 +15678,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15700,7 +15700,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15722,7 +15722,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15744,7 +15744,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15766,7 +15766,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(3, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15788,7 +15788,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(5, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15810,7 +15810,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -15832,7 +15832,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(20, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -16206,7 +16206,7 @@ Project [NOT (cast(cast(1 as decimal(20,0)) as double) = cast(cast(1 as string) -- !query SELECT cast(1 as decimal(3, 0)) <> cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -16228,7 +16228,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) <> cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -16250,7 +16250,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) <> cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -16272,7 +16272,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) <> cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -16338,7 +16338,7 @@ Project [NOT (cast(1 as decimal(20,0)) = cast(cast(1 as boolean) as decimal(20,0 -- !query SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -16360,7 +16360,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -16382,7 +16382,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -16404,7 +16404,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -16426,7 +16426,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -16448,7 +16448,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -16470,7 +16470,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -16492,7 +16492,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/division.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/division.sql.out index 671c5c550cc6..a034c22ffcda 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/division.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/division.sql.out @@ -98,7 +98,7 @@ Project [(cast(cast(1 as tinyint) as double) / cast(cast(cast(1 as string) as do -- !query SELECT cast(1 as tinyint) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -120,7 +120,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as tinyint) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -142,7 +142,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -164,7 +164,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -274,7 +274,7 @@ Project [(cast(cast(1 as smallint) as double) / cast(cast(cast(1 as string) as d -- !query SELECT cast(1 as smallint) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -296,7 +296,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as smallint) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -318,7 +318,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -340,7 +340,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -450,7 +450,7 @@ Project [(cast(cast(1 as int) as double) / cast(cast(cast(1 as string) as double -- !query SELECT cast(1 as int) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -472,7 +472,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as int) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -494,7 +494,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as int) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -516,7 +516,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as int) / cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -626,7 +626,7 @@ Project [(cast(cast(1 as bigint) as double) / cast(cast(cast(1 as string) as dou -- !query SELECT cast(1 as bigint) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -648,7 +648,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as bigint) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -670,7 +670,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -692,7 +692,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -802,7 +802,7 @@ Project [(cast(cast(1 as float) as double) / cast(cast(cast(1 as string) as doub -- !query SELECT cast(1 as float) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -824,7 +824,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as float) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -846,7 +846,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as float) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -868,7 +868,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as float) / cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -978,7 +978,7 @@ Project [(cast(1 as double) / cast(cast(1 as string) as double)) AS (CAST(1 AS D -- !query SELECT cast(1 as double) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1000,7 +1000,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as double) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1022,7 +1022,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as double) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1044,7 +1044,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as double) / cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1154,7 +1154,7 @@ Project [(cast(cast(1 as decimal(10,0)) as double) / cast(cast(1 as string) as d -- !query SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1176,7 +1176,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1198,7 +1198,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1220,7 +1220,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1330,7 +1330,7 @@ Project [(cast(cast(1 as string) as double) / cast(cast(1 as string) as double)) -- !query SELECT cast(1 as string) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1352,7 +1352,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as string) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1374,7 +1374,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as string) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1396,7 +1396,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as string) / cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1418,7 +1418,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) / cast(1 as tinyint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1440,7 +1440,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) / cast(1 as smallint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1462,7 +1462,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) / cast(1 as int) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1484,7 +1484,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) / cast(1 as bigint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1506,7 +1506,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) / cast(1 as float) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1528,7 +1528,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) / cast(1 as double) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1550,7 +1550,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1572,7 +1572,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) / cast(1 as string) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1594,7 +1594,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", @@ -1616,7 +1616,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1638,7 +1638,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1660,7 +1660,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) / cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1682,7 +1682,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) / cast(1 as tinyint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1704,7 +1704,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) / cast(1 as smallint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1726,7 +1726,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) / cast(1 as int) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1748,7 +1748,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) / cast(1 as bigint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1770,7 +1770,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) / cast(1 as float) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1792,7 +1792,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) / cast(1 as double) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1814,7 +1814,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) / cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1836,7 +1836,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) / cast(1 as string) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1858,7 +1858,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1880,7 +1880,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", @@ -1902,7 +1902,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1924,7 +1924,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1946,7 +1946,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as tinyint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1968,7 +1968,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as smallint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1990,7 +1990,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as int) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2012,7 +2012,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as bigint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2034,7 +2034,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as float) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2056,7 +2056,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as double) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2078,7 +2078,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2100,7 +2100,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as string) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2122,7 +2122,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2144,7 +2144,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2166,7 +2166,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", @@ -2188,7 +2188,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2210,7 +2210,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as tinyint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2232,7 +2232,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as smallint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2254,7 +2254,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as int) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2276,7 +2276,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as bigint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2298,7 +2298,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as float) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2320,7 +2320,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as double) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2342,7 +2342,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2364,7 +2364,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as string) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2386,7 +2386,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2408,7 +2408,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2430,7 +2430,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2452,7 +2452,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/ifCoercion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/ifCoercion.sql.out index c93a21f3f37c..edee343b5925 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/ifCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/ifCoercion.sql.out @@ -98,7 +98,7 @@ Project [if (true) cast(cast(1 as tinyint) as string) else cast(2 as string) AS -- !query SELECT IF(true, cast(1 as tinyint), cast('2' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -120,7 +120,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as tinyint), cast(2 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -142,7 +142,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -164,7 +164,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -274,7 +274,7 @@ Project [if (true) cast(cast(1 as smallint) as string) else cast(2 as string) AS -- !query SELECT IF(true, cast(1 as smallint), cast('2' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -296,7 +296,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as smallint), cast(2 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -318,7 +318,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -340,7 +340,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -450,7 +450,7 @@ Project [if (true) cast(cast(1 as int) as string) else cast(2 as string) AS (IF( -- !query SELECT IF(true, cast(1 as int), cast('2' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -472,7 +472,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as int), cast(2 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -494,7 +494,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as int), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -516,7 +516,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as int), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -626,7 +626,7 @@ Project [if (true) cast(cast(1 as bigint) as string) else cast(2 as string) AS ( -- !query SELECT IF(true, cast(1 as bigint), cast('2' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -648,7 +648,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as bigint), cast(2 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -670,7 +670,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -692,7 +692,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -802,7 +802,7 @@ Project [if (true) cast(cast(1 as float) as string) else cast(2 as string) AS (I -- !query SELECT IF(true, cast(1 as float), cast('2' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -824,7 +824,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as float), cast(2 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -846,7 +846,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as float), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -868,7 +868,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as float), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -978,7 +978,7 @@ Project [if (true) cast(cast(1 as double) as string) else cast(2 as string) AS ( -- !query SELECT IF(true, cast(1 as double), cast('2' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1000,7 +1000,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as double), cast(2 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1022,7 +1022,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as double), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1044,7 +1044,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as double), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1154,7 +1154,7 @@ Project [if (true) cast(cast(1 as decimal(10,0)) as string) else cast(2 as strin -- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast('2' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1176,7 +1176,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1198,7 +1198,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1220,7 +1220,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1330,7 +1330,7 @@ Project [if (true) cast(1 as string) else cast(2 as string) AS (IF(true, CAST(1 -- !query SELECT IF(true, cast(1 as string), cast('2' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1352,7 +1352,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as string), cast(2 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1396,7 +1396,7 @@ Project [if (true) cast(1 as string) else cast(cast(2017-12-11 09:30:00 as date) -- !query SELECT IF(true, cast('1' as binary), cast(2 as tinyint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1418,7 +1418,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('1' as binary), cast(2 as smallint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1440,7 +1440,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('1' as binary), cast(2 as int)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1462,7 +1462,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('1' as binary), cast(2 as bigint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1484,7 +1484,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('1' as binary), cast(2 as float)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1506,7 +1506,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('1' as binary), cast(2 as double)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1528,7 +1528,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('1' as binary), cast(2 as decimal(10, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1550,7 +1550,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('1' as binary), cast(2 as string)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1583,7 +1583,7 @@ Project [if (true) cast(1 as binary) else cast(2 as binary) AS (IF(true, CAST(1 -- !query SELECT IF(true, cast('1' as binary), cast(2 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1605,7 +1605,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('1' as binary), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1627,7 +1627,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('1' as binary), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1649,7 +1649,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as boolean), cast(2 as tinyint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1671,7 +1671,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as boolean), cast(2 as smallint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1693,7 +1693,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as boolean), cast(2 as int)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1715,7 +1715,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as boolean), cast(2 as bigint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1737,7 +1737,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as boolean), cast(2 as float)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1759,7 +1759,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as boolean), cast(2 as double)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1781,7 +1781,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as boolean), cast(2 as decimal(10, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1803,7 +1803,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as boolean), cast(2 as string)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1825,7 +1825,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as boolean), cast('2' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1858,7 +1858,7 @@ Project [if (true) cast(1 as boolean) else cast(2 as boolean) AS (IF(true, CAST( -- !query SELECT IF(true, cast(1 as boolean), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1880,7 +1880,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast(1 as boolean), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1902,7 +1902,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as tinyint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1924,7 +1924,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as smallint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1946,7 +1946,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as int)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1968,7 +1968,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as bigint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1990,7 +1990,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as float)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2012,7 +2012,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as double)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2034,7 +2034,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as decimal(10, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2067,7 +2067,7 @@ Project [if (true) cast(cast(2017-12-12 09:30:00.0 as timestamp) as string) else -- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast('2' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2089,7 +2089,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2133,7 +2133,7 @@ Project [if (true) cast(2017-12-12 09:30:00.0 as timestamp) else cast(cast(2017- -- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as tinyint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2155,7 +2155,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as smallint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2177,7 +2177,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as int)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2199,7 +2199,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as bigint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2221,7 +2221,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as float)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2243,7 +2243,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as double)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2265,7 +2265,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as decimal(10, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2298,7 +2298,7 @@ Project [if (true) cast(cast(2017-12-12 09:30:00 as date) as string) else cast(2 -- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast('2' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2320,7 +2320,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/inConversion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/inConversion.sql.out index ae8c3f647933..b248ec599729 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/inConversion.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/inConversion.sql.out @@ -98,7 +98,7 @@ Project [cast(cast(1 as tinyint) as string) IN (cast(cast(1 as string) as string -- !query SELECT cast(1 as tinyint) in (cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -120,7 +120,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as tinyint) in (cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -142,7 +142,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -164,7 +164,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -274,7 +274,7 @@ Project [cast(cast(1 as smallint) as string) IN (cast(cast(1 as string) as strin -- !query SELECT cast(1 as smallint) in (cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -296,7 +296,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as smallint) in (cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -318,7 +318,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -340,7 +340,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -450,7 +450,7 @@ Project [cast(cast(1 as int) as string) IN (cast(cast(1 as string) as string)) A -- !query SELECT cast(1 as int) in (cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -472,7 +472,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as int) in (cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -494,7 +494,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as int) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -516,7 +516,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as int) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -626,7 +626,7 @@ Project [cast(cast(1 as bigint) as string) IN (cast(cast(1 as string) as string) -- !query SELECT cast(1 as bigint) in (cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -648,7 +648,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as bigint) in (cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -670,7 +670,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as bigint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -692,7 +692,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as bigint) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -802,7 +802,7 @@ Project [cast(cast(1 as float) as string) IN (cast(cast(1 as string) as string)) -- !query SELECT cast(1 as float) in (cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -824,7 +824,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as float) in (cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -846,7 +846,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as float) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -868,7 +868,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as float) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -978,7 +978,7 @@ Project [cast(cast(1 as double) as string) IN (cast(cast(1 as string) as string) -- !query SELECT cast(1 as double) in (cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1000,7 +1000,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as double) in (cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1022,7 +1022,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as double) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1044,7 +1044,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as double) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1154,7 +1154,7 @@ Project [cast(cast(1 as decimal(10,0)) as string) IN (cast(cast(1 as string) as -- !query SELECT cast(1 as decimal(10, 0)) in (cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1176,7 +1176,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1198,7 +1198,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1220,7 +1220,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1330,7 +1330,7 @@ Project [cast(1 as string) IN (cast(1 as string)) AS (CAST(1 AS STRING) IN (CAST -- !query SELECT cast(1 as string) in (cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1352,7 +1352,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as string) in (cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1396,7 +1396,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(2017-12-11 09:30:00 as -- !query SELECT cast('1' as binary) in (cast(1 as tinyint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1418,7 +1418,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast(1 as smallint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1440,7 +1440,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast(1 as int)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1462,7 +1462,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast(1 as bigint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1484,7 +1484,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast(1 as float)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1506,7 +1506,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast(1 as double)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1528,7 +1528,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast(1 as decimal(10, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1550,7 +1550,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast(1 as string)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1583,7 +1583,7 @@ Project [cast(1 as binary) IN (cast(1 as binary)) AS (CAST(1 AS BINARY) IN (CAST -- !query SELECT cast('1' as binary) in (cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1605,7 +1605,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1627,7 +1627,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1649,7 +1649,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT true in (cast(1 as tinyint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1671,7 +1671,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT true in (cast(1 as smallint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1693,7 +1693,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT true in (cast(1 as int)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1715,7 +1715,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT true in (cast(1 as bigint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1737,7 +1737,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT true in (cast(1 as float)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1759,7 +1759,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT true in (cast(1 as double)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1781,7 +1781,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT true in (cast(1 as decimal(10, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1803,7 +1803,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT true in (cast(1 as string)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1825,7 +1825,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT true in (cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1858,7 +1858,7 @@ Project [true IN (cast(1 as boolean)) AS (true IN (CAST(1 AS BOOLEAN)))#x] -- !query SELECT true in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1880,7 +1880,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT true in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1902,7 +1902,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as tinyint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1924,7 +1924,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as smallint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1946,7 +1946,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as int)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1968,7 +1968,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as bigint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1990,7 +1990,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as float)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2012,7 +2012,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as double)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2034,7 +2034,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as decimal(10, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2067,7 +2067,7 @@ Project [cast(cast(2017-12-12 09:30:00.0 as timestamp) as string) IN (cast(cast( -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2089,7 +2089,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2133,7 +2133,7 @@ Project [cast(cast(2017-12-12 09:30:00.0 as timestamp) as timestamp) IN (cast(ca -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as tinyint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2155,7 +2155,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as smallint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2177,7 +2177,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as int)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2199,7 +2199,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as bigint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2221,7 +2221,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as float)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2243,7 +2243,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as double)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2265,7 +2265,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as decimal(10, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2298,7 +2298,7 @@ Project [cast(cast(2017-12-12 09:30:00 as date) as string) IN (cast(cast(2 as st -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2320,7 +2320,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2452,7 +2452,7 @@ Project [cast(cast(1 as tinyint) as string) IN (cast(cast(1 as tinyint) as strin -- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2474,7 +2474,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2496,7 +2496,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2518,7 +2518,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2628,7 +2628,7 @@ Project [cast(cast(1 as smallint) as string) IN (cast(cast(1 as smallint) as str -- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2650,7 +2650,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2672,7 +2672,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2694,7 +2694,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2804,7 +2804,7 @@ Project [cast(cast(1 as int) as string) IN (cast(cast(1 as int) as string),cast( -- !query SELECT cast(1 as int) in (cast(1 as int), cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2826,7 +2826,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as int) in (cast(1 as int), cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2848,7 +2848,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as int) in (cast(1 as int), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2870,7 +2870,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as int) in (cast(1 as int), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2980,7 +2980,7 @@ Project [cast(cast(1 as bigint) as string) IN (cast(cast(1 as bigint) as string) -- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3002,7 +3002,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3024,7 +3024,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3046,7 +3046,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3156,7 +3156,7 @@ Project [cast(cast(1 as float) as string) IN (cast(cast(1 as float) as string),c -- !query SELECT cast(1 as float) in (cast(1 as float), cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3178,7 +3178,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as float) in (cast(1 as float), cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3200,7 +3200,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as float) in (cast(1 as float), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3222,7 +3222,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as float) in (cast(1 as float), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3332,7 +3332,7 @@ Project [cast(cast(1 as double) as string) IN (cast(cast(1 as double) as string) -- !query SELECT cast(1 as double) in (cast(1 as double), cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3354,7 +3354,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as double) in (cast(1 as double), cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3376,7 +3376,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as double) in (cast(1 as double), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3398,7 +3398,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as double) in (cast(1 as double), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3508,7 +3508,7 @@ Project [cast(cast(1 as decimal(10,0)) as string) IN (cast(cast(1 as decimal(10, -- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3530,7 +3530,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3552,7 +3552,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3574,7 +3574,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3684,7 +3684,7 @@ Project [cast(1 as string) IN (cast(1 as string),cast(1 as string)) AS (CAST(1 A -- !query SELECT cast(1 as string) in (cast(1 as string), cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3706,7 +3706,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3750,7 +3750,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string) -- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as tinyint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3772,7 +3772,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as smallint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3794,7 +3794,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as int)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3816,7 +3816,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as bigint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3838,7 +3838,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as float)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3860,7 +3860,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as double)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3882,7 +3882,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as decimal(10, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3904,7 +3904,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as string)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3937,7 +3937,7 @@ Project [cast(1 as binary) IN (cast(1 as binary),cast(1 as binary)) AS (CAST(1 A -- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3959,7 +3959,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast('1' as binary), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3981,7 +3981,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) in (cast('1' as binary), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4003,7 +4003,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as tinyint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4025,7 +4025,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as smallint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4047,7 +4047,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as int)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4069,7 +4069,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as bigint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4091,7 +4091,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as float)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4113,7 +4113,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as double)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4135,7 +4135,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as decimal(10, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4157,7 +4157,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as string)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4179,7 +4179,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4212,7 +4212,7 @@ Project [cast(1 as boolean) IN (cast(1 as boolean),cast(1 as boolean)) AS (CAST( -- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4234,7 +4234,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4256,7 +4256,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as tinyint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4278,7 +4278,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as smallint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4300,7 +4300,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as int)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4322,7 +4322,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as bigint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4344,7 +4344,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as float)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4366,7 +4366,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as double)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4388,7 +4388,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as decimal(10, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4421,7 +4421,7 @@ Project [cast(cast(2017-12-12 09:30:00.0 as timestamp) as string) IN (cast(cast( -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4443,7 +4443,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4487,7 +4487,7 @@ Project [cast(cast(2017-12-12 09:30:00.0 as timestamp) as timestamp) IN (cast(ca -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as tinyint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4509,7 +4509,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as smallint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4531,7 +4531,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as int)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4553,7 +4553,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as bigint)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4575,7 +4575,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as float)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4597,7 +4597,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as double)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4619,7 +4619,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as decimal(10, 0))) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4652,7 +4652,7 @@ Project [cast(cast(2017-12-12 09:30:00 as date) as string) IN (cast(cast(2017-12 -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4674,7 +4674,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapZipWith.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapZipWith.sql.out index 0ab21df6b65f..bc8909f1c103 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapZipWith.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapZipWith.sql.out @@ -128,7 +128,7 @@ Project [map_zip_with(double_map#x, cast(float_map#x as map), lamb SELECT map_zip_with(decimal_map1, decimal_map2, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", "sqlState" : "42K09", @@ -178,7 +178,7 @@ Project [map_zip_with(cast(decimal_map1#x as map), double_ SELECT map_zip_with(decimal_map2, int_map, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapconcat.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapconcat.sql.out index b6af7ee4ac5c..7d42c1f7ebe5 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapconcat.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapconcat.sql.out @@ -131,7 +131,7 @@ SELECT map_concat(tinyint_map1, array_map1) tm_map FROM various_maps -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -155,7 +155,7 @@ SELECT map_concat(boolean_map1, int_map2) bi_map FROM various_maps -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -179,7 +179,7 @@ SELECT map_concat(int_map1, struct_map2) is_map FROM various_maps -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -203,7 +203,7 @@ SELECT map_concat(struct_map1, array_map2) ma_map FROM various_maps -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -227,7 +227,7 @@ SELECT map_concat(int_map1, array_map2) ms_map FROM various_maps -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/promoteStrings.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/promoteStrings.sql.out index fbfdaa9558b2..d18d070e54e9 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/promoteStrings.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/promoteStrings.sql.out @@ -98,7 +98,7 @@ Project [(cast(1 as double) + cast(1 as double)) AS (1 + 1)#x] -- !query SELECT '1' + cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -120,7 +120,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' + cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -142,7 +142,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' + cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -164,7 +164,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' + cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -276,7 +276,7 @@ Project [(cast(1 as double) - cast(1 as double)) AS (1 - 1)#x] -- !query SELECT '1' - cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -298,7 +298,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' - cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -320,7 +320,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' - cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -443,7 +443,7 @@ Project [(cast(1 as double) * cast(1 as double)) AS (1 * 1)#x] -- !query SELECT '1' * cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -465,7 +465,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' * cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -487,7 +487,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' * cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -509,7 +509,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' * cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -619,7 +619,7 @@ Project [(cast(1 as double) / cast(1 as double)) AS (1 / 1)#x] -- !query SELECT '1' / cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -641,7 +641,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' / cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -663,7 +663,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -685,7 +685,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' / cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -795,7 +795,7 @@ Project [(cast(1 as double) % cast(1 as double)) AS (1 % 1)#x] -- !query SELECT '1' % cast('1' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -817,7 +817,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' % cast(1 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -839,7 +839,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' % cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -861,7 +861,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT '1' % cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -971,7 +971,7 @@ Project [pmod(cast(1 as double), cast(1 as double)) AS pmod(1, 1)#x] -- !query SELECT pmod('1', cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -993,7 +993,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod('1', cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1015,7 +1015,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod('1', cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1037,7 +1037,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod('1', cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1136,7 +1136,7 @@ Project [(cast(cast(1 as decimal(10,0)) as double) + cast(1 as double)) AS (CAST -- !query SELECT cast('1' as binary) + '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1158,7 +1158,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) + '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1180,7 +1180,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) + '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1202,7 +1202,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) + '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1303,7 +1303,7 @@ Project [(cast(cast(1 as decimal(10,0)) as double) - cast(1 as double)) AS (CAST -- !query SELECT cast('1' as binary) - '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1325,7 +1325,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) - '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1347,7 +1347,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) - '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1371,7 +1371,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) - '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1472,7 +1472,7 @@ Project [(cast(cast(1 as decimal(10,0)) as double) * cast(1 as double)) AS (CAST -- !query SELECT cast('1' as binary) * '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1494,7 +1494,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) * '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1516,7 +1516,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) * '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1538,7 +1538,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) * '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1637,7 +1637,7 @@ Project [(cast(cast(1 as decimal(10,0)) as double) / cast(1 as double)) AS (CAST -- !query SELECT cast('1' as binary) / '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1659,7 +1659,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) / '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1681,7 +1681,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1703,7 +1703,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) / '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1802,7 +1802,7 @@ Project [(cast(cast(1 as decimal(10,0)) as double) % cast(1 as double)) AS (CAST -- !query SELECT cast('1' as binary) % '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1824,7 +1824,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) % '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1846,7 +1846,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) % '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1868,7 +1868,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-11 09:30:00' as date) % '1' FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1967,7 +1967,7 @@ Project [pmod(cast(cast(1 as decimal(10,0)) as double), cast(1 as double)) AS pm -- !query SELECT pmod(cast('1' as binary), '1') FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1989,7 +1989,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast(1 as boolean), '1') FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2011,7 +2011,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), '1') FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2033,7 +2033,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT pmod(cast('2017-12-11 09:30:00' as date), '1') FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/stringCastAndExpressions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/stringCastAndExpressions.sql.out index 495f996de279..448ab457d395 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/stringCastAndExpressions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/stringCastAndExpressions.sql.out @@ -131,7 +131,7 @@ Project [cast(a#x as binary) AS a#x] -- !query select cast(a as array) from t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", "sqlState" : "42K09", @@ -153,7 +153,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(a as struct) from t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", "sqlState" : "42K09", @@ -175,7 +175,7 @@ org.apache.spark.sql.AnalysisException -- !query select cast(a as map) from t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/widenSetOperationTypes.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/widenSetOperationTypes.sql.out index 75975258e883..ff009f8bd64c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/widenSetOperationTypes.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/widenSetOperationTypes.sql.out @@ -169,7 +169,7 @@ Distinct -- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -194,7 +194,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -219,7 +219,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -244,7 +244,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -428,7 +428,7 @@ Distinct -- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast('2' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -453,7 +453,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -478,7 +478,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -503,7 +503,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -687,7 +687,7 @@ Distinct -- !query SELECT cast(1 as int) FROM t UNION SELECT cast('2' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -712,7 +712,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as int) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -737,7 +737,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -762,7 +762,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -947,7 +947,7 @@ Distinct -- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast('2' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -972,7 +972,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -997,7 +997,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1022,7 +1022,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1207,7 +1207,7 @@ Distinct -- !query SELECT cast(1 as float) FROM t UNION SELECT cast('2' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1232,7 +1232,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as float) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1257,7 +1257,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1282,7 +1282,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1466,7 +1466,7 @@ Distinct -- !query SELECT cast(1 as double) FROM t UNION SELECT cast('2' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1491,7 +1491,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as double) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1516,7 +1516,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1541,7 +1541,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1727,7 +1727,7 @@ Distinct -- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1752,7 +1752,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1777,7 +1777,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1802,7 +1802,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1986,7 +1986,7 @@ Distinct -- !query SELECT cast(1 as string) FROM t UNION SELECT cast('2' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2011,7 +2011,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2076,7 +2076,7 @@ Distinct -- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as tinyint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2101,7 +2101,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as smallint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2126,7 +2126,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as int) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2151,7 +2151,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as bigint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2176,7 +2176,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as float) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2201,7 +2201,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as double) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2226,7 +2226,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2251,7 +2251,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as string) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2295,7 +2295,7 @@ Distinct -- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2320,7 +2320,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2345,7 +2345,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2370,7 +2370,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as tinyint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2395,7 +2395,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as smallint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2420,7 +2420,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as int) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2445,7 +2445,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as bigint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2470,7 +2470,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as float) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2495,7 +2495,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as double) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2520,7 +2520,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2545,7 +2545,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as string) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2570,7 +2570,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast('2' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2614,7 +2614,7 @@ Distinct -- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2639,7 +2639,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2664,7 +2664,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as tinyint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2689,7 +2689,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as smallint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2714,7 +2714,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as int) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2739,7 +2739,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as bigint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2764,7 +2764,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as float) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2789,7 +2789,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as double) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2814,7 +2814,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2859,7 +2859,7 @@ Distinct -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2884,7 +2884,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2948,7 +2948,7 @@ Distinct -- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as tinyint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2973,7 +2973,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as smallint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2998,7 +2998,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as int) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -3023,7 +3023,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as bigint) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -3048,7 +3048,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as float) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -3073,7 +3073,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as double) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -3098,7 +3098,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -3143,7 +3143,7 @@ Distinct -- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2' as binary) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -3168,7 +3168,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/windowFrameCoercion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/windowFrameCoercion.sql.out index 50ef28306167..76595c6cbded 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/windowFrameCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/windowFrameCoercion.sql.out @@ -276,7 +276,7 @@ Project [count(1) OVER (PARTITION BY 1 ORDER BY CAST(1 AS DECIMAL(10,0)) DESC NU -- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", "sqlState" : "42K09", @@ -299,7 +299,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", "sqlState" : "42K09", @@ -322,7 +322,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", "sqlState" : "42K09", @@ -345,7 +345,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as timestamp) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part1.sql.out index 07fc99bb3809..7610adb2076d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part1.sql.out @@ -494,7 +494,7 @@ select (select udf(max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)))) from tenk1 o -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part3.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part3.sql.out index 2529794a929b..14f9c4f9d87a 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part3.sql.out @@ -2,7 +2,7 @@ -- !query select udf(max(min(unique1))) from tenk1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NESTED_AGGREGATE_FUNCTION", "sqlState" : "42607", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out index 83395e9e9b18..02d3facbfaa2 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out @@ -659,7 +659,15 @@ Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(i#x as st -- !query CREATE TABLE t1 (name STRING, n INTEGER) USING parquet -- !query analysis -CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "LOCATION_ALREADY_EXISTS", + "sqlState" : "42710", + "messageParameters" : { + "identifier" : "`spark_catalog`.`default`.`t1`", + "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" + } +} -- !query @@ -677,9 +685,21 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t3`, false -- !query INSERT INTO t1 VALUES ( 'bb', 11 ) -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [name, n] -+- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] - +- LocalRelation [col1#x, col2#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query @@ -733,17 +753,21 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d -- !query SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name) -- !query analysis -Project [name#x, n#x, n#x, n#x] -+- Project [coalesce(name#x, name#x) AS name#x, n#x, n#x, n#x] - +- Join FullOuter, (name#x = name#x) - :- Project [coalesce(name#x, name#x) AS name#x, n#x, n#x] - : +- Join FullOuter, (name#x = name#x) - : :- SubqueryAlias spark_catalog.default.t1 - : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet - : +- SubqueryAlias spark_catalog.default.t2 - : +- Relation spark_catalog.default.t2[name#x,n#x] parquet - +- SubqueryAlias spark_catalog.default.t3 - +- Relation spark_catalog.default.t3[name#x,n#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 16, + "fragment" : "t1" + } ] +} -- !query @@ -871,23 +895,21 @@ NATURAL INNER JOIN NATURAL INNER JOIN (SELECT udf(udf(udf(name))) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3 -- !query analysis -Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] -+- Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] - +- Join Inner, (name#x = name#x) - :- Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x] - : +- Join Inner, (name#x = name#x) - : :- SubqueryAlias s1 - : : +- Project [cast(udf(cast(cast(udf(cast(name#x as string)) as string) as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s1_n#x, 1 AS s1_1#x] - : : +- SubqueryAlias spark_catalog.default.t1 - : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet - : +- SubqueryAlias s2 - : +- Project [cast(udf(cast(name#x as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s2_n#x, 2 AS s2_2#x] - : +- SubqueryAlias spark_catalog.default.t2 - : +- Relation spark_catalog.default.t2[name#x,n#x] parquet - +- SubqueryAlias s3 - +- Project [cast(udf(cast(cast(udf(cast(cast(udf(cast(name#x as string)) as string) as string)) as string) as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s3_n#x, 3 AS s3_2#x] - +- SubqueryAlias spark_catalog.default.t3 - +- Relation spark_catalog.default.t3[name#x,n#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 78, + "stopIndex" : 79, + "fragment" : "t1" + } ] +} -- !query @@ -898,23 +920,21 @@ NATURAL FULL JOIN NATURAL FULL JOIN (SELECT udf(udf(name)) as name, udf(n) as s3_n, udf(3) as s3_2 FROM t3) s3 -- !query analysis -Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] -+- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] - +- Join FullOuter, (name#x = name#x) - :- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x] - : +- Join FullOuter, (name#x = name#x) - : :- SubqueryAlias s1 - : : +- Project [cast(udf(cast(name#x as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s1_n#x, cast(udf(cast(cast(udf(cast(1 as string)) as int) as string)) as int) AS s1_1#x] - : : +- SubqueryAlias spark_catalog.default.t1 - : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet - : +- SubqueryAlias s2 - : +- Project [cast(udf(cast(name#x as string)) as string) AS name#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS s2_n#x, cast(udf(cast(2 as string)) as int) AS s2_2#x] - : +- SubqueryAlias spark_catalog.default.t2 - : +- Relation spark_catalog.default.t2[name#x,n#x] parquet - +- SubqueryAlias s3 - +- Project [cast(udf(cast(cast(udf(cast(name#x as string)) as string) as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s3_n#x, cast(udf(cast(3 as string)) as int) AS s3_2#x] - +- SubqueryAlias spark_catalog.default.t3 - +- Relation spark_catalog.default.t3[name#x,n#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 83, + "stopIndex" : 84, + "fragment" : "t1" + } ] +} -- !query @@ -927,25 +947,21 @@ NATURAL FULL JOIN (SELECT name, udf(udf(n)) as s3_n FROM t3) as s3 ) ss2 -- !query analysis -Project [name#x, cast(udf(cast(cast(udf(cast(s1_n#x as string)) as int) as string)) as int) AS udf(udf(s1_n))#x, cast(udf(cast(s2_n#x as string)) as int) AS udf(s2_n)#x, cast(udf(cast(s3_n#x as string)) as int) AS udf(s3_n)#x] -+- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s2_n#x, s3_n#x] - +- Join FullOuter, (name#x = name#x) - :- SubqueryAlias s1 - : +- Project [name#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS s1_n#x] - : +- SubqueryAlias spark_catalog.default.t1 - : +- Relation spark_catalog.default.t1[name#x,n#x] parquet - +- SubqueryAlias ss2 - +- Project [name#x, s2_n#x, s3_n#x] - +- Project [coalesce(name#x, name#x) AS name#x, s2_n#x, s3_n#x] - +- Join FullOuter, (name#x = name#x) - :- SubqueryAlias s2 - : +- Project [name#x, cast(udf(cast(n#x as string)) as int) AS s2_n#x] - : +- SubqueryAlias spark_catalog.default.t2 - : +- Relation spark_catalog.default.t2[name#x,n#x] parquet - +- SubqueryAlias s3 - +- Project [name#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS s3_n#x] - +- SubqueryAlias spark_catalog.default.t3 - +- Relation spark_catalog.default.t3[name#x,n#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 95, + "stopIndex" : 96, + "fragment" : "t1" + } ] +} -- !query @@ -958,25 +974,21 @@ NATURAL FULL JOIN (SELECT name, udf(n) as s3_n FROM t3) as s3 ) ss2 -- !query analysis -Project [name#x, s1_n#x, s2_n#x, s2_2#x, s3_n#x] -+- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s2_n#x, s2_2#x, s3_n#x] - +- Join FullOuter, (name#x = name#x) - :- SubqueryAlias s1 - : +- Project [name#x, n#x AS s1_n#x] - : +- SubqueryAlias spark_catalog.default.t1 - : +- Relation spark_catalog.default.t1[name#x,n#x] parquet - +- SubqueryAlias ss2 - +- Project [name#x, s2_n#x, s2_2#x, s3_n#x] - +- Project [coalesce(name#x, name#x) AS name#x, s2_n#x, s2_2#x, s3_n#x] - +- Join FullOuter, (name#x = name#x) - :- SubqueryAlias s2 - : +- Project [name#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS s2_n#x, 2 AS s2_2#x] - : +- SubqueryAlias spark_catalog.default.t2 - : +- Relation spark_catalog.default.t2[name#x,n#x] parquet - +- SubqueryAlias s3 - +- Project [name#x, cast(udf(cast(n#x as string)) as int) AS s3_n#x] - +- SubqueryAlias spark_catalog.default.t3 - +- Relation spark_catalog.default.t3[name#x,n#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 45, + "fragment" : "t1" + } ] +} -- !query @@ -986,16 +998,21 @@ FULL JOIN (SELECT name, 2 as s2_n FROM t2) as s2 ON (udf(udf(s1_n)) = udf(s2_n)) -- !query analysis -Project [name#x, cast(udf(cast(s1_n#x as string)) as int) AS udf(s1_n)#x, name#x, cast(udf(cast(cast(udf(cast(s2_n#x as string)) as int) as string)) as int) AS udf(udf(s2_n))#x] -+- Join FullOuter, (cast(udf(cast(cast(udf(cast(s1_n#x as string)) as int) as string)) as int) = cast(udf(cast(s2_n#x as string)) as int)) - :- SubqueryAlias s1 - : +- Project [name#x, cast(udf(cast(n#x as string)) as int) AS s1_n#x] - : +- SubqueryAlias spark_catalog.default.t1 - : +- Relation spark_catalog.default.t1[name#x,n#x] parquet - +- SubqueryAlias s2 - +- Project [name#x, 2 AS s2_n#x] - +- SubqueryAlias spark_catalog.default.t2 - +- Relation spark_catalog.default.t2[name#x,n#x] parquet +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 93, + "stopIndex" : 94, + "fragment" : "t1" + } ] +} -- !query @@ -1590,8 +1607,14 @@ Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] -- !query DROP TABLE t1 -- !query analysis -DropTable false, false -+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1 +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`t1`" + } +} -- !query @@ -3008,7 +3031,7 @@ org.apache.spark.sql.AnalysisException select * from int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(y.f1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3065,7 +3088,7 @@ Project [q1#xL, q2#xL, f1#x, ff#x] select udf(t1.uunique1) from tenk1 t1 join tenk2 t2 on t1.two = udf(t2.two) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3087,7 +3110,7 @@ org.apache.spark.sql.AnalysisException select udf(udf(t2.uunique1)) from tenk1 t1 join tenk2 t2 on udf(t1.two) = t2.two -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3109,7 +3132,7 @@ org.apache.spark.sql.AnalysisException select udf(uunique1) from tenk1 t1 join tenk2 t2 on udf(t1.two) = udf(t2.two) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3433,7 +3456,7 @@ Project [q1#xL, q2#xL, bq1#xL, cq1#xL, least(outer(a.q1), q1, q1)#xL] -- !query select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -3453,7 +3476,7 @@ org.apache.spark.sql.AnalysisException -- !query select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -3473,7 +3496,7 @@ org.apache.spark.sql.AnalysisException -- !query select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -3493,7 +3516,7 @@ org.apache.spark.sql.AnalysisException -- !query select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_having.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_having.sql.out index 065a0c65a407..5f7e92a62f30 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_having.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_having.sql.out @@ -146,7 +146,7 @@ Project [udf(min(udf(a)))#x, udf(udf(max(a)))#x] -- !query SELECT udf(a) FROM test_having HAVING udf(min(a)) < udf(max(a)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -163,7 +163,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT 1 AS one FROM test_having HAVING udf(a) > 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_implicit.sql.out index 2377140ad92f..2dac7538cb44 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_implicit.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_implicit.sql.out @@ -110,7 +110,7 @@ Project [udf(count(1))#xL] -- !query SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -323,7 +323,7 @@ Project [udf(count(c))#xL] -- !query SELECT udf(count(udf(a))) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-except-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-except-all.sql.out index 9787bdcc9c35..70b6ec4726f9 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-except-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-except-all.sql.out @@ -199,7 +199,7 @@ SELECT udf(c1) FROM tab1 EXCEPT ALL SELECT array(1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -376,7 +376,7 @@ SELECT k FROM tab3 EXCEPT ALL SELECT k, v FROM tab4 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NUM_COLUMNS_MISMATCH", "sqlState" : "42826", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-analytics.sql.out index 8a9c142bb87e..ae4dbeca9b09 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-analytics.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-analytics.sql.out @@ -203,7 +203,7 @@ Aggregate [course#x, year#x, spark_grouping_id#xL], [cast(udf(cast(course#x as s -- !query SELECT course, udf(year), GROUPING(course) FROM courseSales GROUP BY course, udf(year) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", "queryContext" : [ { @@ -219,7 +219,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT course, udf(year), GROUPING_ID(course, year) FROM courseSales GROUP BY udf(course), year -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", "queryContext" : [ { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-by.sql.out index 6d2d924ce8d4..3f45d866a859 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-by.sql.out @@ -15,7 +15,7 @@ AS testData(a, b), false, true, LocalTempView, true -- !query SELECT udf(a), udf(COUNT(b)) FROM testData -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -56,7 +56,7 @@ Aggregate [a#x], [cast(udf(cast(a#x as string)) as int) AS udf(a)#x, count(cast( -- !query SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -145,7 +145,7 @@ Aggregate [(a#x + b#x)], [cast(udf(cast((a#x + b#x) as string)) as int) AS udf(( -- !query SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -218,7 +218,7 @@ Filter (k#x > 1) -- !query SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "GROUP_BY_AGGREGATE", "sqlState" : "42903", @@ -249,7 +249,7 @@ CreateViewCommand `testDataHasSameNameWithAlias`, SELECT * FROM VALUES -- !query SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -269,7 +269,7 @@ SetCommand (spark.sql.groupByAliases,Some(false)) -- !query SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -377,7 +377,7 @@ Project [udf(udf(1))#x] -- !query SELECT udf(id) FROM range(10) HAVING id > 0 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -547,7 +547,7 @@ Aggregate [cast(udf(cast(cast(udf(cast(k#x as string)) as int) as string)) as in -- !query SELECT every(udf(1)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -571,7 +571,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT some(udf(1S)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -595,7 +595,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT any(udf(1L)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -619,7 +619,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT udf(every("true")) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -731,7 +731,7 @@ Project [cnt#xL] -- !query SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WHERE_CONDITION", "sqlState" : "42903", @@ -752,7 +752,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WHERE_CONDITION", "sqlState" : "42903", @@ -773,7 +773,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WHERE_CONDITION", "sqlState" : "42903", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-intersect-all.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-intersect-all.sql.out index ad026de10ed1..91ba35bad21f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-intersect-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-intersect-all.sql.out @@ -161,7 +161,7 @@ SELECT k, udf(v) FROM tab1 INTERSECT ALL SELECT array(1), udf(2) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -188,7 +188,7 @@ SELECT udf(k) FROM tab1 INTERSECT ALL SELECT udf(k), udf(v) FROM tab2 -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NUM_COLUMNS_MISMATCH", "sqlState" : "42826", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-pivot.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-pivot.sql.out index fa94f77207be..08d11e0ac448 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-pivot.sql.out @@ -346,7 +346,7 @@ PIVOT ( FOR year IN (2012, 2013) ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -396,7 +396,7 @@ PIVOT ( FOR course IN ('dotNET', 'Java') ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NESTED_AGGREGATE_FUNCTION", "sqlState" : "42607", @@ -504,7 +504,7 @@ PIVOT ( FOR year IN (s, 2013) ) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-window.sql.out index c00c55e6b8c7..977996c68471 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-window.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-window.sql.out @@ -68,7 +68,7 @@ Project [udf(val)#x, cate#x, sum(val) OVER (PARTITION BY cate ORDER BY udf(val) SELECT val_long, udf(cate), sum(val_long) OVER(PARTITION BY cate ORDER BY udf(val_long) ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY udf(cate), val_long -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", "sqlState" : "42K09", @@ -219,7 +219,7 @@ Project [val#x, udf(cate)#x, sum(val) OVER (PARTITION BY cate ORDER BY val DESC SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_INVALID_BOUND", "sqlState" : "42K09", @@ -242,7 +242,7 @@ org.apache.spark.sql.AnalysisException SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", "sqlState" : "42K09", @@ -263,7 +263,7 @@ org.apache.spark.sql.AnalysisException SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY udf(val), cate RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_MULTI_ORDER", "sqlState" : "42K09", @@ -285,7 +285,7 @@ org.apache.spark.sql.AnalysisException SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY current_timestamp RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", "sqlState" : "42K09", @@ -308,7 +308,7 @@ org.apache.spark.sql.AnalysisException SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY val RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY udf(cate), val -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_WRONG_COMPARISON", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out index cc2cc8bfc0e2..61a2d69a9bee 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out @@ -140,7 +140,7 @@ Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", "sqlState" : "42K09", @@ -376,7 +376,7 @@ SELECT val_date, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_da RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData ORDER BY cate, val_date -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", "sqlState" : "42K09", @@ -416,7 +416,7 @@ Sort [cate#x ASC NULLS FIRST, val#x ASC NULLS FIRST], true SELECT val, cate, count(val) OVER(PARTITION BY cate ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, val -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_INVALID_BOUND", "sqlState" : "42K09", @@ -439,7 +439,7 @@ org.apache.spark.sql.AnalysisException SELECT val, cate, count(val) OVER(PARTITION BY cate RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", "sqlState" : "42K09", @@ -460,7 +460,7 @@ org.apache.spark.sql.AnalysisException SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val, cate RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_MULTI_ORDER", "sqlState" : "42K09", @@ -482,7 +482,7 @@ org.apache.spark.sql.AnalysisException SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY current_timestamp RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", "sqlState" : "42K09", @@ -505,7 +505,7 @@ org.apache.spark.sql.AnalysisException SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY cate, val -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_WRONG_COMPARISON", "sqlState" : "42K09", @@ -1257,7 +1257,7 @@ SELECT FROM basic_pays -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "_LEGACY_ERROR_TEMP_1004", "messageParameters" : { @@ -1272,7 +1272,7 @@ SELECT FROM basic_pays -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "_LEGACY_ERROR_TEMP_1004", "messageParameters" : { diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out index 91294fffe049..62a74435d9bf 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out @@ -127,7 +127,7 @@ select sort_array(array('b', 'd'), '1') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -153,7 +153,7 @@ select sort_array(array('b', 'd'), cast(NULL as boolean)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -449,7 +449,7 @@ select array_size(map('a', 1, 'b', 2)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -598,7 +598,7 @@ select array_insert(array(1, 2, 3), 3, "4") -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", "sqlState" : "42K09", @@ -664,7 +664,7 @@ select array_compact(id) from values (1) as t(id) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out index 7492391a32e3..355b65f853fe 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out @@ -610,7 +610,7 @@ SELECT HEX(CAST(CAST(123 AS byte) AS binary)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", @@ -636,7 +636,7 @@ SELECT HEX(CAST(CAST(-123 AS byte) AS binary)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", @@ -662,7 +662,7 @@ SELECT HEX(CAST(123S AS binary)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", @@ -688,7 +688,7 @@ SELECT HEX(CAST(-123S AS binary)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", @@ -714,7 +714,7 @@ SELECT HEX(CAST(123 AS binary)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", @@ -740,7 +740,7 @@ SELECT HEX(CAST(-123 AS binary)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", @@ -766,7 +766,7 @@ SELECT HEX(CAST(123L AS binary)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", @@ -792,7 +792,7 @@ SELECT HEX(CAST(-123L AS binary)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/date.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/date.sql.out index dee745995ea8..a30030bbaca9 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/date.sql.out @@ -378,7 +378,7 @@ select date_add('2011-11-11', 1L) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -404,7 +404,7 @@ select date_add('2011-11-11', 1.0) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -430,7 +430,7 @@ select date_add('2011-11-11', 1E1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -553,7 +553,7 @@ select date_sub('2011-11-11', 1L) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -579,7 +579,7 @@ select date_sub('2011-11-11', 1.0) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -605,7 +605,7 @@ select date_sub('2011-11-11', 1E1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -728,7 +728,7 @@ select date '2011-11-11' + 1E1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -834,7 +834,7 @@ select date'2011-11-11' + '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -860,7 +860,7 @@ select '1' + date'2011-11-11' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out index 557f8684ac6b..861afcc7b100 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out @@ -132,7 +132,7 @@ SELECT 1 FROM `hello` -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -169,7 +169,7 @@ ALTER TABLE `not_exist` ADD COLUMN not_exist int -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -191,7 +191,7 @@ ALTER TABLE not_exist ADD COLUMN `not_exist` int -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -213,7 +213,7 @@ SELECT 1 AS `not_exist` FROM `not_exist` -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -235,7 +235,7 @@ SELECT 1 FROM not_exist AS X(`hello`) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out index b2b44777b9ef..2854e09aab6b 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out @@ -4,7 +4,7 @@ SELECT 1 FROM "not_exist" -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -41,7 +41,7 @@ ALTER TABLE "not_exist" ADD COLUMN not_exist int -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -63,7 +63,7 @@ ALTER TABLE not_exist ADD COLUMN "not_exist" int -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -85,7 +85,7 @@ SELECT 1 AS "not_exist" FROM not_exist -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -107,7 +107,7 @@ SELECT 1 FROM not_exist AS X("hello") -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -175,7 +175,7 @@ SELECT 1 FROM `hello` -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -212,7 +212,7 @@ ALTER TABLE `not_exist` ADD COLUMN not_exist int -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -234,7 +234,7 @@ ALTER TABLE not_exist ADD COLUMN `not_exist` int -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -256,7 +256,7 @@ SELECT 1 AS `not_exist` FROM `not_exist` -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -278,7 +278,7 @@ SELECT 1 FROM not_exist AS X(`hello`) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -346,7 +346,7 @@ SELECT "hello" -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", 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 c5c73002a1a9..9eb4a4766df8 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 @@ -291,7 +291,7 @@ select '2' / interval 2 second -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -315,7 +315,7 @@ select '2' / interval 2 year -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -425,7 +425,7 @@ select 2 / interval '2' year -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -449,7 +449,7 @@ select 2 / interval '2' hour -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -473,7 +473,7 @@ select null / interval '2' year -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -497,7 +497,7 @@ select null / interval '2' hour -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1857,7 +1857,7 @@ select -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1897,7 +1897,7 @@ select interval '2' year + '3-3' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1921,7 +1921,7 @@ select interval '2' year - '4' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2003,7 +2003,7 @@ select interval '2' year + str from interval_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2027,7 +2027,7 @@ select interval '2' year - str from interval_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2101,7 +2101,7 @@ select interval '2-2' year to month + interval '3' day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2127,7 +2127,7 @@ select interval '3' day + interval '2-2' year to month -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2153,7 +2153,7 @@ select interval '2-2' year to month - interval '3' day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2179,7 +2179,7 @@ select interval '3' day - interval '2-2' year to month -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2203,7 +2203,7 @@ select 1 - interval '2' second -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2229,7 +2229,7 @@ select 1 + interval '2' month -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2253,7 +2253,7 @@ select interval '2' second + 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2279,7 +2279,7 @@ select interval '2' month - 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3309,7 +3309,7 @@ SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3333,7 +3333,7 @@ SELECT INTERVAL '1' DAY < '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3357,7 +3357,7 @@ SELECT INTERVAL '1' DAY = '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3381,7 +3381,7 @@ SELECT INTERVAL '1' DAY > '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3405,7 +3405,7 @@ SELECT '1' < INTERVAL '1' DAY -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3429,7 +3429,7 @@ SELECT '1' = INTERVAL '1' DAY -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3453,7 +3453,7 @@ SELECT '1' > INTERVAL '1' DAY -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3477,7 +3477,7 @@ SELECT INTERVAL '1' YEAR < '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3501,7 +3501,7 @@ SELECT INTERVAL '1' YEAR = '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3525,7 +3525,7 @@ SELECT INTERVAL '1' YEAR > '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3549,7 +3549,7 @@ SELECT '1' < INTERVAL '1' YEAR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3573,7 +3573,7 @@ SELECT '1' = INTERVAL '1' YEAR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3597,7 +3597,7 @@ SELECT '1' > INTERVAL '1' YEAR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3637,7 +3637,7 @@ SELECT array(INTERVAL 1 MONTH, INTERVAL 20 DAYS) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3677,7 +3677,7 @@ SELECT coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3749,7 +3749,7 @@ SELECT div(INTERVAL '1' MONTH, INTERVAL '-1' DAY) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index 56f114970034..85bcc2713ff5 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -535,7 +535,7 @@ select +date '1999-01-01' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -561,7 +561,7 @@ select +timestamp '1999-01-01' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -595,7 +595,7 @@ select +map(1, 2) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -621,7 +621,7 @@ select +array(1,2) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -647,7 +647,7 @@ select +named_struct('a', 1, 'b', 'spark') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -673,7 +673,7 @@ select +X'1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -699,7 +699,7 @@ select -date '1999-01-01' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -725,7 +725,7 @@ select -timestamp '1999-01-01' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -751,7 +751,7 @@ select -x'2379ACFe' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out index c4c9d13af991..71f5491287c3 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out @@ -68,7 +68,7 @@ select map_contains_key(map('1', 'a', '2', 'b'), 1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", "sqlState" : "42K09", @@ -94,7 +94,7 @@ select map_contains_key(map(1, 'a', 2, 'b'), '1') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out index 3e72daf6dfc2..e55e3fe2da55 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out @@ -1628,7 +1628,7 @@ select to_binary('abc', 1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", "sqlState" : "42K09", @@ -1654,7 +1654,7 @@ select to_binary('abc', 'invalidFormat') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", "sqlState" : "42K09", @@ -1688,7 +1688,7 @@ SELECT to_binary('abc', fmtField) FROM fmtTable -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", 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 7433fcd52b9c..51314079a8d9 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 @@ -763,7 +763,7 @@ select timestamp'2011-11-11 11:11:11' + '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", @@ -787,7 +787,7 @@ select '1' + timestamp'2011-11-11 11:11:11' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", @@ -811,7 +811,7 @@ select timestamp'2011-11-11 11:11:11' + null -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -835,7 +835,7 @@ select null + timestamp'2011-11-11 11:11:11' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/try_arithmetic.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/try_arithmetic.sql.out index cf29eff19fbb..414198b19645 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/try_arithmetic.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/try_arithmetic.sql.out @@ -210,7 +210,7 @@ SELECT try_add(interval 2 year, interval 2 second) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/array.sql.out b/sql/core/src/test/resources/sql-tests/results/array.sql.out index 58d25f674ea9..6c87afccea20 100644 --- a/sql/core/src/test/resources/sql-tests/results/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/array.sql.out @@ -127,7 +127,7 @@ select sort_array(array('b', 'd'), '1') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -153,7 +153,7 @@ select sort_array(array('b', 'd'), cast(NULL as boolean)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -330,7 +330,7 @@ select array_size(map('a', 1, 'b', 2)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -479,7 +479,7 @@ select array_insert(array(1, 2, 3), 3, "4") -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", "sqlState" : "42K09", @@ -545,7 +545,7 @@ select array_compact(id) from values (1) as t(id) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out b/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out index 3407e72e6fd2..6120132dd085 100644 --- a/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out @@ -148,7 +148,7 @@ select bit_count("bit count") -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -174,7 +174,7 @@ select bit_count('a') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out index 888e8a942891..7681fa95e179 100644 --- a/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out @@ -513,34 +513,37 @@ create table char_part(c1 char(5), c2 char(2), v1 varchar(6), v2 varchar(2)) usi -- !query schema struct<> -- !query output - +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "LOCATION_ALREADY_EXISTS", + "sqlState" : "42710", + "messageParameters" : { + "identifier" : "`spark_catalog`.`default`.`char_part`", + "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/char_part'" + } +} -- !query desc formatted char_part -- !query schema -struct +struct<> -- !query output -c1 char(5) -v1 varchar(6) -v2 varchar(2) -c2 char(2) -# Partition Information -# col_name data_type comment -v2 varchar(2) -c2 char(2) - -# Detailed Table Information -Catalog spark_catalog -Database default -Table char_part -Created Time [not included in comparison] -Last Access [not included in comparison] -Created By [not included in comparison] -Type MANAGED -Provider parquet -Location [not included in comparison]/{warehouse_dir}/char_part -Partition Provider Catalog +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 24, + "fragment" : "char_part" + } ] +} -- !query @@ -548,34 +551,43 @@ alter table char_part add partition (v2='ke', c2='nt') location 'loc1' -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "char_part" + } ] +} -- !query desc formatted char_part -- !query schema -struct +struct<> -- !query output -c1 char(5) -v1 varchar(6) -v2 varchar(2) -c2 char(2) -# Partition Information -# col_name data_type comment -v2 varchar(2) -c2 char(2) - -# Detailed Table Information -Catalog spark_catalog -Database default -Table char_part -Created Time [not included in comparison] -Last Access [not included in comparison] -Created By [not included in comparison] -Type MANAGED -Provider parquet -Location [not included in comparison]/{warehouse_dir}/char_part -Partition Provider Catalog +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 24, + "fragment" : "char_part" + } ] +} -- !query @@ -583,42 +595,43 @@ alter table char_part partition (v2='ke') rename to partition (v2='nt') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_1232", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "partitionColumnNames" : "v2, c2", - "specKeys" : "v2", - "tableName" : "`spark_catalog`.`default`.`char_part`" - } + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "char_part" + } ] } -- !query desc formatted char_part -- !query schema -struct +struct<> -- !query output -c1 char(5) -v1 varchar(6) -v2 varchar(2) -c2 char(2) -# Partition Information -# col_name data_type comment -v2 varchar(2) -c2 char(2) - -# Detailed Table Information -Catalog spark_catalog -Database default -Table char_part -Created Time [not included in comparison] -Last Access [not included in comparison] -Created By [not included in comparison] -Type MANAGED -Provider parquet -Location [not included in comparison]/{warehouse_dir}/char_part -Partition Provider Catalog +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 24, + "fragment" : "char_part" + } ] +} -- !query @@ -626,34 +639,43 @@ alter table char_part partition (v2='ke', c2='nt') set location 'loc2' -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "char_part" + } ] +} -- !query desc formatted char_part -- !query schema -struct +struct<> -- !query output -c1 char(5) -v1 varchar(6) -v2 varchar(2) -c2 char(2) -# Partition Information -# col_name data_type comment -v2 varchar(2) -c2 char(2) - -# Detailed Table Information -Catalog spark_catalog -Database default -Table char_part -Created Time [not included in comparison] -Last Access [not included in comparison] -Created By [not included in comparison] -Type MANAGED -Provider parquet -Location [not included in comparison]/{warehouse_dir}/char_part -Partition Provider Catalog +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 24, + "fragment" : "char_part" + } ] +} -- !query @@ -661,34 +683,43 @@ MSCK REPAIR TABLE char_part -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 27, + "fragment" : "char_part" + } ] +} -- !query desc formatted char_part -- !query schema -struct +struct<> -- !query output -c1 char(5) -v1 varchar(6) -v2 varchar(2) -c2 char(2) -# Partition Information -# col_name data_type comment -v2 varchar(2) -c2 char(2) - -# Detailed Table Information -Catalog spark_catalog -Database default -Table char_part -Created Time [not included in comparison] -Last Access [not included in comparison] -Created By [not included in comparison] -Type MANAGED -Provider parquet -Location [not included in comparison]/{warehouse_dir}/char_part -Partition Provider Catalog +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`char_part`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 24, + "fragment" : "char_part" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/column-resolution-aggregate.sql.out b/sql/core/src/test/resources/sql-tests/results/column-resolution-aggregate.sql.out index e8ab766751c4..bc4942be7860 100644 --- a/sql/core/src/test/resources/sql-tests/results/column-resolution-aggregate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/column-resolution-aggregate.sql.out @@ -20,7 +20,7 @@ SELECT max(a) AS b, b FROM v1 GROUP BY k -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -45,7 +45,7 @@ SELECT a AS k FROM v1 GROUP BY k -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -61,7 +61,7 @@ SELECT x FROM v2 GROUP BY all -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -77,7 +77,7 @@ SELECT a AS all, b FROM v1 GROUP BY all -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", 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 b3e62faa3b34..e87c0eb3d84d 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 @@ -283,7 +283,7 @@ SELECT db1.t1.i1 FROM t1, mydb2.t1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -322,7 +322,7 @@ SELECT mydb1.t1 FROM t1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -367,7 +367,7 @@ SELECT t1 FROM mydb1.t1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -398,7 +398,7 @@ SELECT mydb1.t1.i1 FROM t1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out index 50d7f406b026..d91d70cd36ba 100644 --- a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out @@ -42,7 +42,7 @@ select from_csv('1', 'a InvalidType') -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.AnalysisException { "errorClass" : "PARSE_SYNTAX_ERROR", "sqlState" : "42601", @@ -170,7 +170,7 @@ select schema_of_csv(null) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL", "sqlState" : "42K09", @@ -201,7 +201,7 @@ SELECT schema_of_csv(csvField) FROM csvTable -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out index 4ef176e35291..b79d8b1afb0d 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out @@ -44,7 +44,7 @@ SELECT * FROM cte -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -259,7 +259,7 @@ SELECT * FROM t2 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -310,7 +310,7 @@ SELECT * FROM ( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -343,7 +343,7 @@ SELECT * FROM ( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -378,7 +378,7 @@ SELECT * FROM ( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out index 7c683c9df07a..efb0bf48383c 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out @@ -44,7 +44,7 @@ SELECT * FROM cte -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -379,7 +379,7 @@ SELECT * FROM ( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -414,7 +414,7 @@ SELECT * FROM ( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out index 17abf4cbc893..a93bcb759376 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out @@ -44,7 +44,7 @@ SELECT * FROM cte -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -315,7 +315,7 @@ SELECT * FROM ( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -350,7 +350,7 @@ SELECT * FROM ( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/results/cte.sql.out b/sql/core/src/test/resources/sql-tests/results/cte.sql.out index cf585b8d8840..cbff3e8d08b1 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte.sql.out @@ -20,7 +20,7 @@ WITH s AS (SELECT 1 FROM s) SELECT * FROM s -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -43,7 +43,7 @@ SELECT * FROM r -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -75,7 +75,7 @@ WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -208,7 +208,7 @@ SELECT 2 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -333,7 +333,7 @@ select id from cte -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -358,7 +358,7 @@ select 2 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -407,7 +407,7 @@ SELECT * FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -524,7 +524,7 @@ with cte AS (SELECT * FROM va) SELECT * FROM cte -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -546,7 +546,7 @@ with cte as (select * from cte) select * from cte -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/results/date.sql.out b/sql/core/src/test/resources/sql-tests/results/date.sql.out index 322d916f2829..bae2b4291222 100644 --- a/sql/core/src/test/resources/sql-tests/results/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date.sql.out @@ -332,7 +332,7 @@ select date_add('2011-11-11', 1L) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -358,7 +358,7 @@ select date_add('2011-11-11', 1.0) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -384,7 +384,7 @@ select date_add('2011-11-11', 1E1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -497,7 +497,7 @@ select date_sub('2011-11-11', 1L) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -523,7 +523,7 @@ select date_sub('2011-11-11', 1.0) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -549,7 +549,7 @@ select date_sub('2011-11-11', 1E1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -630,7 +630,7 @@ select date_add('2011-11-11', int_str) from date_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -656,7 +656,7 @@ select date_sub('2011-11-11', int_str) from date_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -698,7 +698,7 @@ select date '2011-11-11' + 1E1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -756,7 +756,7 @@ select date '2001-10-01' - '2001-09-28' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -814,7 +814,7 @@ select date '2001-09-28' - date_str from date_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -840,7 +840,7 @@ select date'2011-11-11' + '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -866,7 +866,7 @@ select '1' + date'2011-11-11' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", 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 a824fd96c116..1642e10ea416 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 @@ -332,7 +332,7 @@ select date_add('2011-11-11', 1L) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -358,7 +358,7 @@ select date_add('2011-11-11', 1.0) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -384,7 +384,7 @@ select date_add('2011-11-11', 1E1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -497,7 +497,7 @@ select date_sub('2011-11-11', 1L) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -523,7 +523,7 @@ select date_sub('2011-11-11', 1.0) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -549,7 +549,7 @@ select date_sub('2011-11-11', 1E1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -630,7 +630,7 @@ select date_add('2011-11-11', int_str) from date_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -656,7 +656,7 @@ select date_sub('2011-11-11', int_str) from date_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -698,7 +698,7 @@ select date '2011-11-11' + 1E1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -756,7 +756,7 @@ select date '2001-10-01' - '2001-09-28' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -814,7 +814,7 @@ select date '2001-09-28' - date_str from date_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -840,7 +840,7 @@ select date'2011-11-11' + '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -866,7 +866,7 @@ select '1' + date'2011-11-11' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1828,7 +1828,7 @@ select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1854,7 +1854,7 @@ select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1904,7 +1904,7 @@ select str - timestamp'2011-11-11 11:11:11' from ts_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1930,7 +1930,7 @@ select timestamp'2011-11-11 11:11:11' - str from ts_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1956,7 +1956,7 @@ select timestamp'2011-11-11 11:11:11' + '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1980,7 +1980,7 @@ select '1' + timestamp'2011-11-11 11:11:11' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2004,7 +2004,7 @@ select timestamp'2011-11-11 11:11:11' + null -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2028,7 +2028,7 @@ select null + timestamp'2011-11-11 11:11:11' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out b/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out index 557f8684ac6b..861afcc7b100 100644 --- a/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out @@ -132,7 +132,7 @@ SELECT 1 FROM `hello` -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -169,7 +169,7 @@ ALTER TABLE `not_exist` ADD COLUMN not_exist int -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -191,7 +191,7 @@ ALTER TABLE not_exist ADD COLUMN `not_exist` int -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -213,7 +213,7 @@ SELECT 1 AS `not_exist` FROM `not_exist` -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -235,7 +235,7 @@ SELECT 1 FROM not_exist AS X(`hello`) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/results/except-all.sql.out b/sql/core/src/test/resources/sql-tests/results/except-all.sql.out index f487a92e678f..437a5553d385 100644 --- a/sql/core/src/test/resources/sql-tests/results/except-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/except-all.sql.out @@ -137,7 +137,7 @@ SELECT array(1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -227,7 +227,7 @@ SELECT k, v FROM tab4 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NUM_COLUMNS_MISMATCH", "sqlState" : "42826", diff --git a/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out index d9e1a25b5747..4286034b9284 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out @@ -464,7 +464,7 @@ SELECT course, year, GROUPING(course) FROM courseSales GROUP BY course, year -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", "queryContext" : [ { @@ -482,7 +482,7 @@ SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY course, -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", "queryContext" : [ { diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-all-duckdb.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-all-duckdb.sql.out index 80d73eceb1da..30b5846ceff8 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-all-duckdb.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-all-duckdb.sql.out @@ -55,7 +55,7 @@ SELECT (g+i)%2 + SUM(i), SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", "sqlState" : "42803", @@ -101,7 +101,7 @@ SELECT c0 FROM (SELECT 1 c0, 1 c1 UNION ALL SELECT 1, 2) t0 GROUP BY ALL HAVING -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out index 0da833261694..9bc5fe0cc379 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out @@ -56,7 +56,7 @@ SELECT i + SUM(f) FROM stuff GROUP BY ALL -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", "sqlState" : "42803", @@ -101,7 +101,7 @@ SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING i > 2 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -124,7 +124,7 @@ SELECT SUM(i) si FROM stuff GROUP BY ALL ORDER BY i DESC -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-all.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-all.sql.out index bf6394e31c8f..bf3c1a3c21b8 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-all.sql.out @@ -193,7 +193,7 @@ select id + count(*) from data group by all -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", "sqlState" : "42803", @@ -212,7 +212,7 @@ select (id + id) / 2 + count(*) * 2 from data group by all -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", "sqlState" : "42803", @@ -253,7 +253,7 @@ select (select count(*) from data d1 where d1.country = d2.country) + count(id) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", "sqlState" : "42803", @@ -272,7 +272,7 @@ select non_exist from data group by all -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", 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 232abb1dd23c..6557b18cfcf1 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 @@ -47,7 +47,7 @@ SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -237,7 +237,7 @@ SELECT a, COUNT(b) FILTER (WHERE a != 2) FROM testData GROUP BY b -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -724,7 +724,7 @@ SELECT a + 2, COUNT(b) FILTER (WHERE b IN (1, 2)) FROM testData GROUP BY a + 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", 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 d652cbfa4937..acdf8d5a8547 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 @@ -14,7 +14,7 @@ SELECT a, COUNT(b) FROM testData -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -52,7 +52,7 @@ SELECT a, COUNT(b) FROM testData GROUP BY b -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -123,7 +123,7 @@ SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -198,7 +198,7 @@ SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -221,7 +221,7 @@ SELECT COUNT(b) AS k FROM testData GROUP BY k -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "GROUP_BY_AGGREGATE", "sqlState" : "42903", @@ -252,7 +252,7 @@ SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -276,7 +276,7 @@ SELECT a AS k, COUNT(b) FROM testData GROUP BY k -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -362,7 +362,7 @@ SELECT id FROM range(10) HAVING id > 0 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -406,7 +406,7 @@ SELECT 1 FROM range(10) HAVING MAX(id) > 0 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WHERE_CONDITION", "sqlState" : "42903", @@ -550,7 +550,7 @@ SELECT every(1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -576,7 +576,7 @@ SELECT some(1S) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -602,7 +602,7 @@ SELECT any(1L) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -628,7 +628,7 @@ SELECT every("true") -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -654,7 +654,7 @@ SELECT bool_and(1.0) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -680,7 +680,7 @@ SELECT bool_or(1.0D) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -817,7 +817,7 @@ SELECT count(*) FROM test_agg WHERE count(*) > 1L -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WHERE_CONDITION", "sqlState" : "42903", @@ -840,7 +840,7 @@ SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WHERE_CONDITION", "sqlState" : "42903", @@ -863,7 +863,7 @@ SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WHERE_CONDITION", "sqlState" : "42903", 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 61d9523da6d4..5849acd094fb 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 @@ -165,7 +165,7 @@ SELECT c1 FROM (values (1,2), (3,2)) t(c1, c2) GROUP BY GROUPING SETS (()) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", 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 438164bf2e33..6eaba0b4119c 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 @@ -34,7 +34,7 @@ SELECT count(k) FROM hav GROUP BY v HAVING v = array(1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/hll.sql.out b/sql/core/src/test/resources/sql-tests/results/hll.sql.out index c8a2e9a2faf9..c38a57cf8497 100644 --- a/sql/core/src/test/resources/sql-tests/results/hll.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/hll.sql.out @@ -12,15 +12,37 @@ CREATE TABLE t1 USING JSON AS VALUES (0), (1), (2), (2), (2), (3), (4) as tab(co -- !query schema struct<> -- !query output - +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "LOCATION_ALREADY_EXISTS", + "sqlState" : "42710", + "messageParameters" : { + "identifier" : "`spark_catalog`.`default`.`t1`", + "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" + } +} -- !query SELECT hll_sketch_estimate(hll_sketch_agg(col)) AS result FROM t1 -- !query schema -struct +struct<> -- !query output -5 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 64, + "stopIndex" : 65, + "fragment" : "t1" + } ] +} -- !query @@ -77,7 +99,7 @@ FROM VALUES (ARRAY(1, 2)), (ARRAY(3, 4)) tab(col) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -190,7 +212,7 @@ SELECT hll_union(1, 2) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out index 9410ccc5e542..8eabb74da97b 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out @@ -1024,7 +1024,7 @@ SELECT IDENTIFIER('t').c1 FROM VALUES(1) AS T(c1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out b/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out index 10dfb51358f5..44f95f225abe 100644 --- a/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out @@ -94,7 +94,7 @@ SELECT array(1), 2 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -123,7 +123,7 @@ SELECT k, v FROM tab2 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NUM_COLUMNS_MISMATCH", "sqlState" : "42826", 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 2f8185f78d8d..fe15ade94178 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 @@ -189,7 +189,7 @@ select '2' / interval 2 second -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -213,7 +213,7 @@ select '2' / interval 2 year -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -323,7 +323,7 @@ select 2 / interval '2' year -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -347,7 +347,7 @@ select 2 / interval '2' hour -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -371,7 +371,7 @@ select null / interval '2' year -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -395,7 +395,7 @@ select null / interval '2' hour -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1738,7 +1738,7 @@ select -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1778,7 +1778,7 @@ select interval '2' year + '3-3' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1802,7 +1802,7 @@ select interval '2' year - '4' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1850,7 +1850,7 @@ select interval '2' year + str from interval_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1874,7 +1874,7 @@ select interval '2' year - str from interval_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1914,7 +1914,7 @@ select interval '2-2' year to month + interval '3' day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1940,7 +1940,7 @@ select interval '3' day + interval '2-2' year to month -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1966,7 +1966,7 @@ select interval '2-2' year to month - interval '3' day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1992,7 +1992,7 @@ select interval '3' day - interval '2-2' year to month -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2016,7 +2016,7 @@ select 1 - interval '2' second -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2042,7 +2042,7 @@ select 1 + interval '2' month -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2066,7 +2066,7 @@ select interval '2' second + 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2092,7 +2092,7 @@ select interval '2' month - 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3122,7 +3122,7 @@ SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3146,7 +3146,7 @@ SELECT INTERVAL '1' DAY < '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3170,7 +3170,7 @@ SELECT INTERVAL '1' DAY = '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3194,7 +3194,7 @@ SELECT INTERVAL '1' DAY > '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3218,7 +3218,7 @@ SELECT '1' < INTERVAL '1' DAY -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3242,7 +3242,7 @@ SELECT '1' = INTERVAL '1' DAY -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3266,7 +3266,7 @@ SELECT '1' > INTERVAL '1' DAY -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3290,7 +3290,7 @@ SELECT INTERVAL '1' YEAR < '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3314,7 +3314,7 @@ SELECT INTERVAL '1' YEAR = '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3338,7 +3338,7 @@ SELECT INTERVAL '1' YEAR > '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3362,7 +3362,7 @@ SELECT '1' < INTERVAL '1' YEAR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3386,7 +3386,7 @@ SELECT '1' = INTERVAL '1' YEAR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3410,7 +3410,7 @@ SELECT '1' > INTERVAL '1' YEAR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3450,7 +3450,7 @@ SELECT array(INTERVAL 1 MONTH, INTERVAL 20 DAYS) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3490,7 +3490,7 @@ SELECT coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3562,7 +3562,7 @@ SELECT div(INTERVAL '1' MONTH, INTERVAL '-1' DAY) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", 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 ddee595372b0..0bb83be0f03d 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 @@ -317,7 +317,7 @@ SELECT * FROM t1 JOIN LATERAL (SELECT t1.c1 AS a, t2.c1 AS b) s JOIN t2 ON s.b = -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -355,7 +355,7 @@ SELECT * FROM t1, LATERAL (SELECT c1 + c2 + rand(0) AS c3) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_DETERMINISTIC_LATERAL_SUBQUERIES", "sqlState" : "0A000", @@ -377,7 +377,7 @@ SELECT * FROM t1, LATERAL (SELECT rand(0) FROM t2) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_DETERMINISTIC_LATERAL_SUBQUERIES", "sqlState" : "0A000", @@ -399,7 +399,7 @@ SELECT * FROM t1 JOIN LATERAL (SELECT * FROM t2) s ON t1.c1 + rand(0) = s.c1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.LATERAL_JOIN_CONDITION_NON_DETERMINISTIC", "sqlState" : "0A000", @@ -476,7 +476,7 @@ SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.c1 + t2.c1)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -498,7 +498,7 @@ SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1), LATERAL (SELECT c2)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -538,7 +538,7 @@ SELECT * FROM t1, LATERAL (SELECT c1, (SELECT SUM(c2) FROM t2 WHERE c1 = t1.c1)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -860,7 +860,7 @@ SELECT * FROM t1 JOIN LATERAL -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -1165,7 +1165,7 @@ SELECT * FROM t1 JOIN LATERAL -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", @@ -1193,7 +1193,7 @@ SELECT * FROM t1 JOIN LATERAL -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -1344,7 +1344,7 @@ SELECT * FROM t1 JOIN LATERAL (SELECT sum(c1) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -1755,7 +1755,7 @@ SELECT t.* FROM t1, LATERAL stack(c1, c2) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out index 253a255fc165..efa8012dcf80 100644 --- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out @@ -172,7 +172,7 @@ select from_json('{"a":1}', 'a InvalidType') -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.AnalysisException { "errorClass" : "PARSE_SYNTAX_ERROR", "sqlState" : "42601", @@ -494,7 +494,7 @@ select schema_of_json(null) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL", "sqlState" : "42K09", @@ -525,7 +525,7 @@ SELECT schema_of_json(jsonField) FROM jsonTable -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", @@ -558,7 +558,7 @@ select json_array_length(2) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -706,7 +706,7 @@ select json_object_keys(200) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/limit.sql.out b/sql/core/src/test/resources/sql-tests/results/limit.sql.out index e07ee71aa882..c29feb656d87 100644 --- a/sql/core/src/test/resources/sql-tests/results/limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/limit.sql.out @@ -49,7 +49,7 @@ SELECT * FROM testdata LIMIT -1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_NEGATIVE", "messageParameters" : { @@ -72,7 +72,7 @@ SELECT * FROM testData TABLESAMPLE (-1 ROWS) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_NEGATIVE", "messageParameters" : { @@ -103,7 +103,7 @@ SELECT * FROM testdata LIMIT CAST(NULL AS INT) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_NULL", "messageParameters" : { @@ -125,7 +125,7 @@ SELECT * FROM testdata LIMIT key > 3 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", "messageParameters" : { @@ -147,7 +147,7 @@ SELECT * FROM testdata LIMIT true -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.DATA_TYPE", "messageParameters" : { @@ -163,7 +163,7 @@ SELECT * FROM testdata LIMIT 'a' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.DATA_TYPE", "messageParameters" : { diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 56f114970034..85bcc2713ff5 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -535,7 +535,7 @@ select +date '1999-01-01' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -561,7 +561,7 @@ select +timestamp '1999-01-01' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -595,7 +595,7 @@ select +map(1, 2) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -621,7 +621,7 @@ select +array(1,2) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -647,7 +647,7 @@ select +named_struct('a', 1, 'b', 'spark') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -673,7 +673,7 @@ select +X'1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -699,7 +699,7 @@ select -date '1999-01-01' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -725,7 +725,7 @@ select -timestamp '1999-01-01' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -751,7 +751,7 @@ select -x'2379ACFe' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/map.sql.out b/sql/core/src/test/resources/sql-tests/results/map.sql.out index c4c9d13af991..71f5491287c3 100644 --- a/sql/core/src/test/resources/sql-tests/results/map.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/map.sql.out @@ -68,7 +68,7 @@ select map_contains_key(map('1', 'a', '2', 'b'), 1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", "sqlState" : "42K09", @@ -94,7 +94,7 @@ select map_contains_key(map(1, 'a', 2, 'b'), '1') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/mask-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/mask-functions.sql.out index d5345de064c6..c00e89da6cb3 100644 --- a/sql/core/src/test/resources/sql-tests/results/mask-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/mask-functions.sql.out @@ -44,7 +44,7 @@ SELECT mask('AbCD123-@$#', 'Qa', 'qa', 'da', 'oa') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", "sqlState" : "42K09", @@ -435,7 +435,7 @@ SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 'a') as t(c1, rep -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", @@ -460,7 +460,7 @@ SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 'ABC') as t(c1, r -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", @@ -485,7 +485,7 @@ SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 123) as t(c1, rep -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -511,7 +511,7 @@ SELECT mask('abcd-EFGH-8765-4321', 'A', 'w', '') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", "sqlState" : "42K09", @@ -534,7 +534,7 @@ SELECT mask('abcd-EFGH-8765-4321', 'A', 'abc') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/named-function-arguments.sql.out b/sql/core/src/test/resources/sql-tests/results/named-function-arguments.sql.out index 3b223cc0e152..77c15b56c8da 100644 --- a/sql/core/src/test/resources/sql-tests/results/named-function-arguments.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/named-function-arguments.sql.out @@ -183,7 +183,7 @@ SELECT * FROM explode(collection => TABLE(v)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", 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 a8c7ef556fbb..38791e6741fc 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 @@ -248,7 +248,7 @@ SELECT nt2.k FROM (SELECT * FROM nt1 natural join nt2) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out b/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out index ece6dbef1605..ff72dcd91551 100644 --- a/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out @@ -4,7 +4,15 @@ create table t1(a int, b int, c int) using parquet -- !query schema struct<> -- !query output - +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "LOCATION_ALREADY_EXISTS", + "sqlState" : "42710", + "messageParameters" : { + "identifier" : "`spark_catalog`.`default`.`t1`", + "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" + } +} -- !query @@ -12,7 +20,21 @@ insert into t1 values(1,0,0) -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query @@ -20,7 +42,21 @@ insert into t1 values(2,0,1) -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query @@ -28,7 +64,21 @@ insert into t1 values(3,1,0) -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query @@ -36,7 +86,21 @@ insert into t1 values(4,1,1) -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query @@ -44,7 +108,21 @@ insert into t1 values(5,null,0) -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query @@ -52,7 +130,21 @@ insert into t1 values(6,null,1) -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query @@ -60,252 +152,461 @@ insert into t1 values(7,null,null) -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query select a, b+c from t1 -- !query schema -struct +struct<> -- !query output -1 0 -2 1 -3 1 -4 2 -5 NULL -6 NULL -7 NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 20, + "stopIndex" : 21, + "fragment" : "t1" + } ] +} -- !query select a+10, b*0 from t1 -- !query schema -struct<(a + 10):int,(b * 0):int> +struct<> -- !query output -11 0 -12 0 -13 0 -14 0 -15 NULL -16 NULL -17 NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 24, + "fragment" : "t1" + } ] +} -- !query select distinct b from t1 -- !query schema -struct +struct<> -- !query output -0 -1 -NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 25, + "fragment" : "t1" + } ] +} -- !query select b from t1 union select b from t1 -- !query schema -struct +struct<> -- !query output -0 -1 -NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 16, + "fragment" : "t1" + } ] +} -- !query select a+20, case b when c then 1 else 0 end from t1 -- !query schema -struct<(a + 20):int,CASE WHEN (b = c) THEN 1 ELSE 0 END:int> +struct<> -- !query output -21 1 -22 0 -23 0 -24 1 -25 0 -26 0 -27 0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 52, + "fragment" : "t1" + } ] +} -- !query select a+30, case c when b then 1 else 0 end from t1 -- !query schema -struct<(a + 30):int,CASE WHEN (c = b) THEN 1 ELSE 0 END:int> +struct<> -- !query output -31 1 -32 0 -33 0 -34 1 -35 0 -36 0 -37 0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 52, + "fragment" : "t1" + } ] +} -- !query select a+40, case when b<>0 then 1 else 0 end from t1 -- !query schema -struct<(a + 40):int,CASE WHEN (NOT (b = 0)) THEN 1 ELSE 0 END:int> +struct<> -- !query output -41 0 -42 0 -43 1 -44 1 -45 0 -46 0 -47 0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 52, + "stopIndex" : 53, + "fragment" : "t1" + } ] +} -- !query select a+50, case when not b<>0 then 1 else 0 end from t1 -- !query schema -struct<(a + 50):int,CASE WHEN (NOT (NOT (b = 0))) THEN 1 ELSE 0 END:int> +struct<> -- !query output -51 1 -52 1 -53 0 -54 0 -55 0 -56 0 -57 0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 56, + "stopIndex" : 57, + "fragment" : "t1" + } ] +} -- !query select a+60, case when b<>0 and c<>0 then 1 else 0 end from t1 -- !query schema -struct<(a + 60):int,CASE WHEN ((NOT (b = 0)) AND (NOT (c = 0))) THEN 1 ELSE 0 END:int> +struct<> -- !query output -61 0 -62 0 -63 0 -64 1 -65 0 -66 0 -67 0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 61, + "stopIndex" : 62, + "fragment" : "t1" + } ] +} -- !query select a+70, case when not (b<>0 and c<>0) then 1 else 0 end from t1 -- !query schema -struct<(a + 70):int,CASE WHEN (NOT ((NOT (b = 0)) AND (NOT (c = 0)))) THEN 1 ELSE 0 END:int> +struct<> -- !query output -71 1 -72 1 -73 1 -74 0 -75 1 -76 0 -77 0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 67, + "stopIndex" : 68, + "fragment" : "t1" + } ] +} -- !query select a+80, case when b<>0 or c<>0 then 1 else 0 end from t1 -- !query schema -struct<(a + 80):int,CASE WHEN ((NOT (b = 0)) OR (NOT (c = 0))) THEN 1 ELSE 0 END:int> +struct<> -- !query output -81 0 -82 1 -83 1 -84 1 -85 0 -86 1 -87 0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 60, + "stopIndex" : 61, + "fragment" : "t1" + } ] +} -- !query select a+90, case when not (b<>0 or c<>0) then 1 else 0 end from t1 -- !query schema -struct<(a + 90):int,CASE WHEN (NOT ((NOT (b = 0)) OR (NOT (c = 0)))) THEN 1 ELSE 0 END:int> +struct<> -- !query output -91 1 -92 0 -93 0 -94 0 -95 0 -96 0 -97 0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 66, + "stopIndex" : 67, + "fragment" : "t1" + } ] +} -- !query select count(*), count(b), sum(b), avg(b), min(b), max(b) from t1 -- !query schema -struct +struct<> -- !query output -7 4 2 0.5 0 1 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 64, + "stopIndex" : 65, + "fragment" : "t1" + } ] +} -- !query select a+100 from t1 where b<10 -- !query schema -struct<(a + 100):int> +struct<> -- !query output -101 -102 -103 -104 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 20, + "fragment" : "t1" + } ] +} -- !query select a+110 from t1 where not b>10 -- !query schema -struct<(a + 110):int> +struct<> -- !query output -111 -112 -113 -114 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 20, + "fragment" : "t1" + } ] +} -- !query select a+120 from t1 where b<10 OR c=1 -- !query schema -struct<(a + 120):int> +struct<> -- !query output -121 -122 -123 -124 -126 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 20, + "fragment" : "t1" + } ] +} -- !query select a+130 from t1 where b<10 AND c=1 -- !query schema -struct<(a + 130):int> +struct<> -- !query output -132 -134 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 20, + "fragment" : "t1" + } ] +} -- !query select a+140 from t1 where not (b<10 AND c=1) -- !query schema -struct<(a + 140):int> +struct<> -- !query output -141 -143 -145 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 20, + "fragment" : "t1" + } ] +} -- !query select a+150 from t1 where not (c=1 AND b<10) -- !query schema -struct<(a + 150):int> +struct<> -- !query output -151 -153 -155 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 20, + "fragment" : "t1" + } ] +} -- !query select b, c, equal_null(b, c), equal_null(c, b) from t1 -- !query schema -struct +struct<> -- !query output -0 0 true true -0 1 false false -1 0 false false -1 1 true true -NULL 0 false false -NULL 1 false false -NULL NULL true true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 54, + "stopIndex" : 55, + "fragment" : "t1" + } ] +} -- !query @@ -313,4 +614,11 @@ drop table t1 -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`t1`" + } +} diff --git a/sql/core/src/test/resources/sql-tests/results/order-by-all.sql.out b/sql/core/src/test/resources/sql-tests/results/order-by-all.sql.out index 0acfda5a6fde..b0fcfbcb633a 100644 --- a/sql/core/src/test/resources/sql-tests/results/order-by-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/order-by-all.sql.out @@ -184,7 +184,7 @@ from values('Lisa', 'Sales', 10000, 35) as T(name, dept, salary, age) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out b/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out index 2b7a36395bf5..a83eee69fec3 100644 --- a/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out @@ -177,7 +177,7 @@ ORDER BY salary -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -205,7 +205,7 @@ ORDER BY salary -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -232,7 +232,7 @@ ORDER BY salary -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -260,7 +260,7 @@ ORDER BY salary -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -288,7 +288,7 @@ ORDER BY salary -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -315,7 +315,7 @@ ORDER BY salary -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -407,7 +407,7 @@ ORDER BY salary -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -436,7 +436,7 @@ ORDER BY salary -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -464,7 +464,7 @@ ORDER BY salary -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -493,7 +493,7 @@ ORDER BY salary -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -522,7 +522,7 @@ ORDER BY salary -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { @@ -550,7 +550,7 @@ ORDER BY salary -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", "messageParameters" : { 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 75eb7c5ab31a..aaafb19cb979 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 @@ -238,7 +238,7 @@ PIVOT ( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -282,7 +282,7 @@ PIVOT ( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NESTED_AGGREGATE_FUNCTION", "sqlState" : "42607", @@ -364,7 +364,7 @@ PIVOT ( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", 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 fcd31080918e..42cf942add48 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 @@ -507,7 +507,7 @@ from tenk1 o -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out index a971f348cdca..038bcc0283a6 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out @@ -4,7 +4,7 @@ select max(min(unique1)) from tenk1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NESTED_AGGREGATE_FUNCTION", "sqlState" : "42607", 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 468239606c1a..bcd14c72a831 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 @@ -69,7 +69,7 @@ CREATE VIEW key_dependent_view_no_cols AS -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -652,7 +652,7 @@ CREATE VIEW v10_temp AS SELECT * FROM v7_temp -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -674,7 +674,7 @@ CREATE VIEW v11_temp AS SELECT t1.id, t2.a FROM base_table t1, v10_temp t2 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -696,7 +696,7 @@ CREATE VIEW v12_temp AS SELECT true FROM v11_temp -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", 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 44c993b73d78..0e54db5ecaea 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 @@ -1956,7 +1956,15 @@ CREATE TABLE t1 (name STRING, n INTEGER) USING parquet -- !query schema struct<> -- !query output - +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "LOCATION_ALREADY_EXISTS", + "sqlState" : "42710", + "messageParameters" : { + "identifier" : "`spark_catalog`.`default`.`t1`", + "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" + } +} -- !query @@ -1980,7 +1988,21 @@ INSERT INTO t1 VALUES ( 'bb', 11 ) -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query @@ -2034,12 +2056,23 @@ struct<> -- !query SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name) -- !query schema -struct +struct<> -- !query output -bb 11 12 13 -cc NULL 22 23 -dd NULL NULL 33 -ee NULL 42 NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 16, + "fragment" : "t1" + } ] +} -- !query @@ -2131,9 +2164,23 @@ NATURAL INNER JOIN NATURAL INNER JOIN (SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 -- !query schema -struct +struct<> -- !query output -bb 11 1 12 2 13 3 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 55, + "stopIndex" : 56, + "fragment" : "t1" + } ] +} -- !query @@ -2144,12 +2191,23 @@ NATURAL FULL JOIN NATURAL FULL JOIN (SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 -- !query schema -struct +struct<> -- !query output -bb 11 1 12 2 13 3 -cc NULL NULL 22 2 23 3 -dd NULL NULL NULL NULL 33 3 -ee NULL NULL 42 2 NULL NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 55, + "stopIndex" : 56, + "fragment" : "t1" + } ] +} -- !query @@ -2162,12 +2220,23 @@ NATURAL FULL JOIN (SELECT name, n as s3_n FROM t3) as s3 ) ss2 -- !query schema -struct +struct<> -- !query output -bb 11 12 13 -cc NULL 22 23 -dd NULL NULL 33 -ee NULL 42 NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 45, + "fragment" : "t1" + } ] +} -- !query @@ -2180,12 +2249,23 @@ NATURAL FULL JOIN (SELECT name, n as s3_n FROM t3) as s3 ) ss2 -- !query schema -struct +struct<> -- !query output -bb 11 12 2 13 -cc NULL 22 2 23 -dd NULL NULL NULL 33 -ee NULL 42 2 NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 45, + "fragment" : "t1" + } ] +} -- !query @@ -2195,12 +2275,23 @@ FULL JOIN (SELECT name, 2 as s2_n FROM t2) as s2 ON (s1_n = s2_n) -- !query schema -struct +struct<> -- !query output -NULL NULL bb 2 -NULL NULL cc 2 -NULL NULL ee 2 -bb 11 NULL NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 46, + "stopIndex" : 47, + "fragment" : "t1" + } ] +} -- !query @@ -2475,7 +2566,14 @@ DROP TABLE t1 -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`t1`" + } +} -- !query @@ -3272,7 +3370,7 @@ select * from -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3305,7 +3403,7 @@ select t1.uunique1 from -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3329,7 +3427,7 @@ select t2.uunique1 from -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3353,7 +3451,7 @@ select uunique1 from -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3567,7 +3665,7 @@ select f1,g from int4_tbl a, (select f1 as g) ss -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -3589,7 +3687,7 @@ select f1,g from int4_tbl a, (select a.f1 as g) ss -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -3611,7 +3709,7 @@ select f1,g from int4_tbl a cross join (select f1 as g) ss -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -3633,7 +3731,7 @@ select f1,g from int4_tbl a cross join (select a.f1 as g) ss -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out index e76ba10299e9..004b7aa8a689 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out @@ -130,7 +130,7 @@ select * from int8_tbl limit (case when random() < 0.5 then bigint(null) end) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", "messageParameters" : { @@ -152,7 +152,7 @@ select * from int8_tbl offset (case when random() < 0.5 then bigint(null) end) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", "messageParameters" : { diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out index 98159c947a29..7b40f0bcaf23 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out @@ -4712,7 +4712,7 @@ SELECT '' AS to_number_2, to_number('-34,338,492.654,878', '99G999G999D999G999' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_FORMAT.THOUSANDS_SEPS_MUST_BEFORE_DEC", "sqlState" : "42601", @@ -4774,7 +4774,7 @@ SELECT '' AS to_number_15, to_number('123,000','999G') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_FORMAT.CONT_THOUSANDS_SEPS", "sqlState" : "42601", 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 0d25ff9f4714..dc295b0578b0 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 @@ -139,7 +139,7 @@ SELECT a FROM test_having HAVING min(a) < max(a) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -158,7 +158,7 @@ SELECT 1 AS one FROM test_having HAVING a > 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", 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 b1087b796a9e..f0c283cb4036 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 @@ -118,7 +118,7 @@ SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -365,7 +365,7 @@ SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", 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 bc33106321c3..7c920bbd32b3 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 @@ -522,7 +522,7 @@ SELECT q1 FROM int8_tbl EXCEPT SELECT q2 FROM int8_tbl ORDER BY q2 LIMIT 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out index 53f9f6cee8c4..9120a0b0b65b 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out @@ -235,7 +235,15 @@ create table t1 (f1 int, f2 int) using parquet -- !query schema struct<> -- !query output - +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "LOCATION_ALREADY_EXISTS", + "sqlState" : "42710", + "messageParameters" : { + "identifier" : "`spark_catalog`.`default`.`t1`", + "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" + } +} -- !query @@ -243,7 +251,21 @@ insert into t1 values (1,1),(1,2),(2,2) -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query @@ -253,19 +275,19 @@ from t1 where f1 = f2 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", - "sqlState" : "42K09", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "sqlExpr" : "\"(PARTITION BY f1 RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)\"" + "relationName" : "`t1`" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 25, - "stopIndex" : 108, - "fragment" : "(partition by f1\n range between 1 preceding and 1 following)" + "startIndex" : 115, + "stopIndex" : 116, + "fragment" : "t1" } ] } @@ -275,10 +297,23 @@ select f1, sum(f1) over (partition by f1 order by f2 range between 1 preceding and 1 following) from t1 where f1 = f2 -- !query schema -struct +struct<> -- !query output -1 1 -2 2 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 102, + "stopIndex" : 103, + "fragment" : "t1" + } ] +} -- !query @@ -286,10 +321,23 @@ select f1, sum(f1) over (partition by f1, f1 order by f2 range between 2 preceding and 1 preceding) from t1 where f1 = f2 -- !query schema -struct +struct<> -- !query output -1 NULL -2 NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 106, + "stopIndex" : 107, + "fragment" : "t1" + } ] +} -- !query @@ -297,10 +345,23 @@ select f1, sum(f1) over (partition by f1, f2 order by f2 range between 1 following and 2 following) from t1 where f1 = f2 -- !query schema -struct +struct<> -- !query output -1 NULL -2 NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 106, + "stopIndex" : 107, + "fragment" : "t1" + } ] +} -- !query @@ -330,7 +391,7 @@ SELECT * FROM empsalary INNER JOIN tenk1 ON row_number() OVER (ORDER BY salary) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_EXPR_FOR_OPERATOR", "messageParameters" : { @@ -351,7 +412,7 @@ SELECT rank() OVER (ORDER BY 1), count(*) FROM empsalary GROUP BY 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_EXPR_FOR_OPERATOR", "messageParameters" : { @@ -455,7 +516,7 @@ SELECT ntile(0) OVER (ORDER BY ten), ten, four FROM tenk1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", "sqlState" : "42K09", @@ -480,7 +541,7 @@ SELECT nth_value(four, 0) OVER (ORDER BY ten), ten, four FROM tenk1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", "sqlState" : "42K09", @@ -521,4 +582,11 @@ DROP TABLE t1 -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`t1`" + } +} diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out index a3e869422c27..af1c33023a91 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out @@ -215,7 +215,7 @@ SELECT * FROM outermost ORDER BY 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -410,7 +410,7 @@ WITH test AS (SELECT 42) INSERT INTO test VALUES (1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", 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 83a3d2c254be..8df97c723e10 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 @@ -32,7 +32,7 @@ SELECT `(a)?+.+` FROM testData2 WHERE a = 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -55,7 +55,7 @@ SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -78,7 +78,7 @@ SELECT `(a|b)` FROM testData2 WHERE a = 2 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -101,7 +101,7 @@ SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -124,7 +124,7 @@ SELECT SUM(`(a|b)?+.+`) FROM testData2 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -147,7 +147,7 @@ SELECT SUM(`(a)`) FROM testData2 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -381,7 +381,7 @@ SELECT SUM(a) FROM testdata3 GROUP BY `(a)` -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -404,7 +404,7 @@ SELECT SUM(a) FROM testdata3 GROUP BY `(a)?+.+` -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/results/random.sql.out b/sql/core/src/test/resources/sql-tests/results/random.sql.out index dea2c69ba035..8a182a0646ca 100644 --- a/sql/core/src/test/resources/sql-tests/results/random.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/random.sql.out @@ -36,7 +36,7 @@ SELECT rand(1.0) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -94,7 +94,7 @@ SELECT rand('1') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out index 9154b14fbcbf..62a0e5ff917c 100644 --- a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out @@ -406,7 +406,7 @@ SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', -2) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", "sqlState" : "42K09", @@ -431,7 +431,7 @@ SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 0) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out index aa64c529a7eb..9a0d82d3617a 100644 --- a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out @@ -90,7 +90,7 @@ SHOW COLUMNS IN badtable FROM showdb -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -145,7 +145,7 @@ SHOW COLUMNS IN showdb.showcolumn3 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -167,7 +167,7 @@ SHOW COLUMNS IN showcolumn3 FROM showdb -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", @@ -189,7 +189,7 @@ SHOW COLUMNS IN showcolumn4 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out index e3fb58e907a2..5fd9cf2a0a67 100644 --- a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out @@ -1560,7 +1560,7 @@ select to_binary('abc', 1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", "sqlState" : "42K09", @@ -1586,7 +1586,7 @@ select to_binary('abc', 'invalidFormat') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", "sqlState" : "42K09", @@ -1620,7 +1620,7 @@ SELECT to_binary('abc', fmtField) FROM fmtTable -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out index 62f7874a4427..a78af33accd7 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out @@ -373,7 +373,7 @@ WHERE EXISTS (SELECT * -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -403,7 +403,7 @@ WHERE NOT EXISTS (SELECT * -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -433,7 +433,7 @@ WHERE EXISTS (SELECT * -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -463,7 +463,7 @@ WHERE EXISTS (SELECT * -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -493,7 +493,7 @@ WHERE EXISTS (SELECT * -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -523,7 +523,7 @@ WHERE EXISTS (SELECT * -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -553,7 +553,7 @@ WHERE NOT EXISTS (SELECT * -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -583,7 +583,7 @@ WHERE NOT EXISTS (SELECT * -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-basic.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-basic.sql.out index 5c288cc394e4..3a07dbdbf00e 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-basic.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-basic.sql.out @@ -37,7 +37,7 @@ select 1 from tab_a where (a1, b1) not in (select (a2, b2) from tab_b) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_LENGTH_MISMATCH", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out index 8ac2ffdf96e2..1d3cdf517194 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out @@ -604,7 +604,7 @@ WHERE t1a IN (SELECT t2a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -633,7 +633,7 @@ WHERE t1a IN (SELECT t2a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -662,7 +662,7 @@ WHERE t1a IN (SELECT t2a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -691,7 +691,7 @@ WHERE t1a IN (SELECT t2a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -720,7 +720,7 @@ WHERE t1a IN (SELECT t2a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -749,7 +749,7 @@ WHERE t1a IN (SELECT t2a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -778,7 +778,7 @@ WHERE t1a NOT IN (SELECT t2a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -807,7 +807,7 @@ WHERE t1a NOT IN (SELECT t2a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -836,7 +836,7 @@ WHERE t1a NOT IN (SELECT t2a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -865,7 +865,7 @@ WHERE t1a NOT IN (SELECT t2a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -894,7 +894,7 @@ WHERE t1a NOT IN (SELECT t2a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", @@ -923,7 +923,7 @@ WHERE t1a NOT IN (SELECT t2a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", "sqlState" : "0A000", 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 d1eb86a2d337..c1d18933f745 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 @@ -42,7 +42,7 @@ AND t2b = (SELECT max(avg) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -69,7 +69,7 @@ WHERE t1a IN (SELECT min(t2a) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", "messageParameters" : { @@ -154,7 +154,7 @@ WHERE t1a IN (SELECT t2a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", @@ -189,7 +189,7 @@ ON EXISTS (SELECT 1 FROM t2 WHERE t2a > t1a) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out index e8f77ff99c58..52338368f25d 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out @@ -60,7 +60,7 @@ FROM t1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_SUBQUERY_EXPRESSION.SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", "sqlState" : "42823", @@ -88,7 +88,7 @@ FROM t1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_SUBQUERY_EXPRESSION.SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", "sqlState" : "42823", @@ -114,7 +114,7 @@ t1a IN (SELECT t2a, t2b -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_LENGTH_MISMATCH", "sqlState" : "42K09", @@ -144,7 +144,7 @@ WHERE -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_LENGTH_MISMATCH", "sqlState" : "42K09", @@ -175,7 +175,7 @@ WHERE -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_DATA_TYPE_MISMATCH", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out index 02f3f9eae790..ef5d941dc979 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out @@ -614,7 +614,7 @@ SELECT * FROM t0 WHERE t0a < -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out index d21873cd3c8e..2799728d48a6 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out @@ -176,7 +176,7 @@ FROM t0 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", @@ -346,7 +346,7 @@ FROM t0 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", @@ -516,7 +516,7 @@ FROM t0 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", @@ -686,7 +686,7 @@ FROM t0 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", @@ -856,7 +856,7 @@ FROM t0 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", @@ -1026,7 +1026,7 @@ FROM t0 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", "sqlState" : "0A000", 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 01318d93dd6a..22de4faf1ce3 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 @@ -82,7 +82,7 @@ SELECT a AS col1, b AS col2 FROM testData AS t(c, d) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out index 1348110a83a3..1995e9e87b3a 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out @@ -255,7 +255,7 @@ select * from explode(null) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -281,7 +281,7 @@ select * from explode(null) t(c1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -307,7 +307,7 @@ select * from explode(1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -483,7 +483,7 @@ select * from inline(null) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -601,7 +601,7 @@ select * from posexplode(1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -807,7 +807,7 @@ select * from json_tuple('{"a": 1}', 1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE", "sqlState" : "42K09", @@ -830,7 +830,7 @@ select * from json_tuple('{"a": 1}', null) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE", "sqlState" : "42K09", @@ -973,7 +973,7 @@ select * from stack(2, 1, '1.1', 'a', 2, 2.2, 'b') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.STACK_COLUMN_DIFF_TYPES", "sqlState" : "42K09", 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 8a49774f74f4..c28fda32c9ca 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 @@ -631,7 +631,7 @@ select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -657,7 +657,7 @@ select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -707,7 +707,7 @@ select str - timestamp'2011-11-11 11:11:11' from ts_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -733,7 +733,7 @@ select timestamp'2011-11-11 11:11:11' - str from ts_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -759,7 +759,7 @@ select timestamp'2011-11-11 11:11:11' + '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -783,7 +783,7 @@ select '1' + timestamp'2011-11-11 11:11:11' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -807,7 +807,7 @@ select timestamp'2011-11-11 11:11:11' + null -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -831,7 +831,7 @@ select null + timestamp'2011-11-11 11:11:11' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", 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 88ba5314b8ad..06b6818e00ec 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 @@ -778,7 +778,7 @@ select timestamp'2011-11-11 11:11:11' + '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", @@ -802,7 +802,7 @@ select '1' + timestamp'2011-11-11 11:11:11' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", @@ -826,7 +826,7 @@ select timestamp'2011-11-11 11:11:11' + null -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -850,7 +850,7 @@ select null + timestamp'2011-11-11 11:11:11' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", 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 545b49445e42..ff13693e65b7 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 @@ -631,7 +631,7 @@ select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -657,7 +657,7 @@ select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -707,7 +707,7 @@ select str - timestamp'2011-11-11 11:11:11' from ts_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -733,7 +733,7 @@ select timestamp'2011-11-11 11:11:11' - str from ts_view -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -759,7 +759,7 @@ select timestamp'2011-11-11 11:11:11' + '1' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -783,7 +783,7 @@ select '1' + timestamp'2011-11-11 11:11:11' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -807,7 +807,7 @@ select timestamp'2011-11-11 11:11:11' + null -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -831,7 +831,7 @@ select null + timestamp'2011-11-11 11:11:11' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out b/sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out index 60d6750237a4..c706a2607892 100644 --- a/sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out @@ -164,7 +164,7 @@ SELECT try_add(interval 2 year, interval 2 second) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", 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 fe7dd473907c..9b363b15c507 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 @@ -76,7 +76,7 @@ SELECT true = cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -108,7 +108,7 @@ SELECT true = cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -132,7 +132,7 @@ SELECT true = cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -220,7 +220,7 @@ SELECT true <=> cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -252,7 +252,7 @@ SELECT true <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -276,7 +276,7 @@ SELECT true <=> cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -364,7 +364,7 @@ SELECT cast('1' as binary) = true FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -396,7 +396,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) = true FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -420,7 +420,7 @@ SELECT cast('2017-12-11 09:30:00' as date) = true FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -508,7 +508,7 @@ SELECT cast('1' as binary) <=> true FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -540,7 +540,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> true FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -564,7 +564,7 @@ SELECT cast('2017-12-11 09:30:00' as date) <=> true FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -652,7 +652,7 @@ SELECT false = cast('0' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -684,7 +684,7 @@ SELECT false = cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -708,7 +708,7 @@ SELECT false = cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -796,7 +796,7 @@ SELECT false <=> cast('0' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -828,7 +828,7 @@ SELECT false <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -852,7 +852,7 @@ SELECT false <=> cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -940,7 +940,7 @@ SELECT cast('0' as binary) = false FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -972,7 +972,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) = false FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -996,7 +996,7 @@ SELECT cast('2017-12-11 09:30:00' as date) = false FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1084,7 +1084,7 @@ SELECT cast('0' as binary) <=> false FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1116,7 +1116,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> false FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1140,7 +1140,7 @@ SELECT cast('2017-12-11 09:30:00' as date) <=> false FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out index 3d5e9ca07907..7973d1157376 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out @@ -76,7 +76,7 @@ SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2' as binary) END FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -100,7 +100,7 @@ SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as boolean) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -124,7 +124,7 @@ SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00.0' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -148,7 +148,7 @@ SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00' as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -236,7 +236,7 @@ SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2' as binary) END FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -260,7 +260,7 @@ SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as boolean) END FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -284,7 +284,7 @@ SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00.0' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -308,7 +308,7 @@ SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00' a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -396,7 +396,7 @@ SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2' as binary) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -420,7 +420,7 @@ SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as boolean) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -444,7 +444,7 @@ SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00.0' as t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -468,7 +468,7 @@ SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00' as dat -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -556,7 +556,7 @@ SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2' as binary) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -580,7 +580,7 @@ SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as boolean) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -604,7 +604,7 @@ SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00.0' a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -628,7 +628,7 @@ SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00' as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -716,7 +716,7 @@ SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2' as binary) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -740,7 +740,7 @@ SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as boolean) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -764,7 +764,7 @@ SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00.0' as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -788,7 +788,7 @@ SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00' as d -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -876,7 +876,7 @@ SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2' as binary) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -900,7 +900,7 @@ SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as boolean) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -924,7 +924,7 @@ SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00.0' a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -948,7 +948,7 @@ SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00' as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1036,7 +1036,7 @@ SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2' as binary) EN -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1060,7 +1060,7 @@ SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as boolean) END -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1084,7 +1084,7 @@ SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1108,7 +1108,7 @@ SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1196,7 +1196,7 @@ SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2' as binary) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1220,7 +1220,7 @@ SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as boolean) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1260,7 +1260,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as tinyint) END FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1284,7 +1284,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as smallint) END FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1308,7 +1308,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as int) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1332,7 +1332,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as bigint) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1356,7 +1356,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as float) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1380,7 +1380,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as double) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1404,7 +1404,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as decimal(10, 0)) EN -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1428,7 +1428,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as string) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1460,7 +1460,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as boolean) END FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1484,7 +1484,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00.0' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1508,7 +1508,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00' a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1532,7 +1532,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as tinyint) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1556,7 +1556,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as smallint) END FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1580,7 +1580,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as int) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1604,7 +1604,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as bigint) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1628,7 +1628,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as float) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1652,7 +1652,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as double) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1676,7 +1676,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as decimal(10, 0)) END -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1700,7 +1700,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as string) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1724,7 +1724,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2' as binary) END FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1756,7 +1756,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00.0' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1780,7 +1780,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00' as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1804,7 +1804,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1828,7 +1828,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1852,7 +1852,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1876,7 +1876,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1900,7 +1900,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1924,7 +1924,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1948,7 +1948,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1980,7 +1980,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2004,7 +2004,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2044,7 +2044,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as ti -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2068,7 +2068,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as sm -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2092,7 +2092,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as in -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2116,7 +2116,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as bi -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2140,7 +2140,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as fl -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2164,7 +2164,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as do -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2188,7 +2188,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as de -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2220,7 +2220,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2' as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2244,7 +2244,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as bo -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out index dae5de25caf6..72231bd169ed 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out @@ -12,7 +12,7 @@ select cast(1 as tinyint) + interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -38,7 +38,7 @@ select cast(1 as smallint) + interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -64,7 +64,7 @@ select cast(1 as int) + interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -90,7 +90,7 @@ select cast(1 as bigint) + interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -116,7 +116,7 @@ select cast(1 as float) + interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -142,7 +142,7 @@ select cast(1 as double) + interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -168,7 +168,7 @@ select cast(1 as decimal(10, 0)) + interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -210,7 +210,7 @@ select cast('1' as binary) + interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -236,7 +236,7 @@ select cast(1 as boolean) + interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -278,7 +278,7 @@ select interval 2 day + cast(1 as tinyint) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -304,7 +304,7 @@ select interval 2 day + cast(1 as smallint) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -330,7 +330,7 @@ select interval 2 day + cast(1 as int) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -356,7 +356,7 @@ select interval 2 day + cast(1 as bigint) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -382,7 +382,7 @@ select interval 2 day + cast(1 as float) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -408,7 +408,7 @@ select interval 2 day + cast(1 as double) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -434,7 +434,7 @@ select interval 2 day + cast(1 as decimal(10, 0)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -476,7 +476,7 @@ select interval 2 day + cast('1' as binary) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -502,7 +502,7 @@ select interval 2 day + cast(1 as boolean) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -544,7 +544,7 @@ select cast(1 as tinyint) - interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -570,7 +570,7 @@ select cast(1 as smallint) - interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -596,7 +596,7 @@ select cast(1 as int) - interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -622,7 +622,7 @@ select cast(1 as bigint) - interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -648,7 +648,7 @@ select cast(1 as float) - interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -674,7 +674,7 @@ select cast(1 as double) - interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -700,7 +700,7 @@ select cast(1 as decimal(10, 0)) - interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -742,7 +742,7 @@ select cast('1' as binary) - interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -768,7 +768,7 @@ select cast(1 as boolean) - interval 2 day -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", 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 f25a0b45a7be..834bb6b296eb 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 @@ -236,7 +236,7 @@ SELECT cast('1' as binary) + cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -260,7 +260,7 @@ SELECT cast('1' as binary) + cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -284,7 +284,7 @@ SELECT cast('1' as binary) + cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -308,7 +308,7 @@ SELECT cast('1' as binary) + cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -332,7 +332,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(3, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -356,7 +356,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(5, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -380,7 +380,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(10, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -404,7 +404,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(20, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -428,7 +428,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -454,7 +454,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -480,7 +480,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -506,7 +506,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -788,7 +788,7 @@ SELECT cast(1 as decimal(3, 0)) + cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -812,7 +812,7 @@ SELECT cast(1 as decimal(5, 0)) + cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -836,7 +836,7 @@ SELECT cast(1 as decimal(10, 0)) + cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -860,7 +860,7 @@ SELECT cast(1 as decimal(20, 0)) + cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -884,7 +884,7 @@ SELECT cast(1 as decimal(3, 0)) + cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -908,7 +908,7 @@ SELECT cast(1 as decimal(5, 0)) + cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -932,7 +932,7 @@ SELECT cast(1 as decimal(10, 0)) + cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -956,7 +956,7 @@ SELECT cast(1 as decimal(20, 0)) + cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -980,7 +980,7 @@ SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1004,7 +1004,7 @@ SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1028,7 +1028,7 @@ SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1052,7 +1052,7 @@ SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1076,7 +1076,7 @@ SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1102,7 +1102,7 @@ SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1128,7 +1128,7 @@ SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1154,7 +1154,7 @@ SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1404,7 +1404,7 @@ SELECT cast('1' as binary) - cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1428,7 +1428,7 @@ SELECT cast('1' as binary) - cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1452,7 +1452,7 @@ SELECT cast('1' as binary) - cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1476,7 +1476,7 @@ SELECT cast('1' as binary) - cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1500,7 +1500,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(3, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1526,7 +1526,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(5, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1552,7 +1552,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(10, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1578,7 +1578,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(20, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1604,7 +1604,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1630,7 +1630,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1656,7 +1656,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1682,7 +1682,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1964,7 +1964,7 @@ SELECT cast(1 as decimal(3, 0)) - cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1988,7 +1988,7 @@ SELECT cast(1 as decimal(5, 0)) - cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2012,7 +2012,7 @@ SELECT cast(1 as decimal(10, 0)) - cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2036,7 +2036,7 @@ SELECT cast(1 as decimal(20, 0)) - cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2060,7 +2060,7 @@ SELECT cast(1 as decimal(3, 0)) - cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2084,7 +2084,7 @@ SELECT cast(1 as decimal(5, 0)) - cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2108,7 +2108,7 @@ SELECT cast(1 as decimal(10, 0)) - cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2132,7 +2132,7 @@ SELECT cast(1 as decimal(20, 0)) - cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2156,7 +2156,7 @@ SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2182,7 +2182,7 @@ SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2208,7 +2208,7 @@ SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2234,7 +2234,7 @@ SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2260,7 +2260,7 @@ SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2286,7 +2286,7 @@ SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2312,7 +2312,7 @@ SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2338,7 +2338,7 @@ SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -2588,7 +2588,7 @@ SELECT cast('1' as binary) * cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2612,7 +2612,7 @@ SELECT cast('1' as binary) * cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2636,7 +2636,7 @@ SELECT cast('1' as binary) * cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2660,7 +2660,7 @@ SELECT cast('1' as binary) * cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2684,7 +2684,7 @@ SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(3, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2708,7 +2708,7 @@ SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(5, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2732,7 +2732,7 @@ SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(10, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2756,7 +2756,7 @@ SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(20, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2780,7 +2780,7 @@ SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2804,7 +2804,7 @@ SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2828,7 +2828,7 @@ SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2852,7 +2852,7 @@ SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3132,7 +3132,7 @@ SELECT cast(1 as decimal(3, 0)) * cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3156,7 +3156,7 @@ SELECT cast(1 as decimal(5, 0)) * cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3180,7 +3180,7 @@ SELECT cast(1 as decimal(10, 0)) * cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3204,7 +3204,7 @@ SELECT cast(1 as decimal(20, 0)) * cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3228,7 +3228,7 @@ SELECT cast(1 as decimal(3, 0)) * cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3252,7 +3252,7 @@ SELECT cast(1 as decimal(5, 0)) * cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3276,7 +3276,7 @@ SELECT cast(1 as decimal(10, 0)) * cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3300,7 +3300,7 @@ SELECT cast(1 as decimal(20, 0)) * cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3324,7 +3324,7 @@ SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3348,7 +3348,7 @@ SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3372,7 +3372,7 @@ SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3396,7 +3396,7 @@ SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3420,7 +3420,7 @@ SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3444,7 +3444,7 @@ SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3468,7 +3468,7 @@ SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3492,7 +3492,7 @@ SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3740,7 +3740,7 @@ SELECT cast('1' as binary) / cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3764,7 +3764,7 @@ SELECT cast('1' as binary) / cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3788,7 +3788,7 @@ SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3812,7 +3812,7 @@ SELECT cast('1' as binary) / cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3836,7 +3836,7 @@ SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(3, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3860,7 +3860,7 @@ SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(5, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3884,7 +3884,7 @@ SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3908,7 +3908,7 @@ SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(20, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3932,7 +3932,7 @@ SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3956,7 +3956,7 @@ SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -3980,7 +3980,7 @@ SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4004,7 +4004,7 @@ SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4284,7 +4284,7 @@ SELECT cast(1 as decimal(3, 0)) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4308,7 +4308,7 @@ SELECT cast(1 as decimal(5, 0)) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4332,7 +4332,7 @@ SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4356,7 +4356,7 @@ SELECT cast(1 as decimal(20, 0)) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4380,7 +4380,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4404,7 +4404,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4428,7 +4428,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4452,7 +4452,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4476,7 +4476,7 @@ SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4500,7 +4500,7 @@ SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4524,7 +4524,7 @@ SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4548,7 +4548,7 @@ SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4572,7 +4572,7 @@ SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4596,7 +4596,7 @@ SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4620,7 +4620,7 @@ SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4644,7 +4644,7 @@ SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4892,7 +4892,7 @@ SELECT cast('1' as binary) % cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4916,7 +4916,7 @@ SELECT cast('1' as binary) % cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4940,7 +4940,7 @@ SELECT cast('1' as binary) % cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4964,7 +4964,7 @@ SELECT cast('1' as binary) % cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -4988,7 +4988,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(3, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5012,7 +5012,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(5, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5036,7 +5036,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(10, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5060,7 +5060,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(20, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5084,7 +5084,7 @@ SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5108,7 +5108,7 @@ SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5132,7 +5132,7 @@ SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5156,7 +5156,7 @@ SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5436,7 +5436,7 @@ SELECT cast(1 as decimal(3, 0)) % cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5460,7 +5460,7 @@ SELECT cast(1 as decimal(5, 0)) % cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5484,7 +5484,7 @@ SELECT cast(1 as decimal(10, 0)) % cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5508,7 +5508,7 @@ SELECT cast(1 as decimal(20, 0)) % cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5532,7 +5532,7 @@ SELECT cast(1 as decimal(3, 0)) % cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5556,7 +5556,7 @@ SELECT cast(1 as decimal(5, 0)) % cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5580,7 +5580,7 @@ SELECT cast(1 as decimal(10, 0)) % cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5604,7 +5604,7 @@ SELECT cast(1 as decimal(20, 0)) % cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5628,7 +5628,7 @@ SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5652,7 +5652,7 @@ SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5676,7 +5676,7 @@ SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5700,7 +5700,7 @@ SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5724,7 +5724,7 @@ SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5748,7 +5748,7 @@ SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5772,7 +5772,7 @@ SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -5796,7 +5796,7 @@ SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6044,7 +6044,7 @@ SELECT pmod(cast('1' as binary), cast(1 as decimal(3, 0))) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6068,7 +6068,7 @@ SELECT pmod(cast('1' as binary), cast(1 as decimal(5, 0))) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6092,7 +6092,7 @@ SELECT pmod(cast('1' as binary), cast(1 as decimal(10, 0))) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6116,7 +6116,7 @@ SELECT pmod(cast('1' as binary), cast(1 as decimal(20, 0))) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6140,7 +6140,7 @@ SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(3, 0)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6164,7 +6164,7 @@ SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(5, 0)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6188,7 +6188,7 @@ SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(10, 0) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6212,7 +6212,7 @@ SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(20, 0) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6236,7 +6236,7 @@ SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(3, 0))) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6260,7 +6260,7 @@ SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(5, 0))) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6284,7 +6284,7 @@ SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(10, 0))) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6308,7 +6308,7 @@ SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(20, 0))) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6588,7 +6588,7 @@ SELECT pmod(cast(1 as decimal(3, 0)) , cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6612,7 +6612,7 @@ SELECT pmod(cast(1 as decimal(5, 0)) , cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6636,7 +6636,7 @@ SELECT pmod(cast(1 as decimal(10, 0)), cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6660,7 +6660,7 @@ SELECT pmod(cast(1 as decimal(20, 0)), cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6684,7 +6684,7 @@ SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6708,7 +6708,7 @@ SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6732,7 +6732,7 @@ SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6756,7 +6756,7 @@ SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6780,7 +6780,7 @@ SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00.0' as timestamp -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6804,7 +6804,7 @@ SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00.0' as timestamp -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6828,7 +6828,7 @@ SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6852,7 +6852,7 @@ SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00.0' as timestamp -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6876,7 +6876,7 @@ SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00' as date)) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6900,7 +6900,7 @@ SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00' as date)) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6924,7 +6924,7 @@ SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -6948,7 +6948,7 @@ SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00' as date)) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7196,7 +7196,7 @@ SELECT cast('1' as binary) = cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7220,7 +7220,7 @@ SELECT cast('1' as binary) = cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7244,7 +7244,7 @@ SELECT cast('1' as binary) = cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7268,7 +7268,7 @@ SELECT cast('1' as binary) = cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7292,7 +7292,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(3, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7316,7 +7316,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(5, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7340,7 +7340,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(10, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7364,7 +7364,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(20, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7388,7 +7388,7 @@ SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7412,7 +7412,7 @@ SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7436,7 +7436,7 @@ SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7460,7 +7460,7 @@ SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7740,7 +7740,7 @@ SELECT cast(1 as decimal(3, 0)) = cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7764,7 +7764,7 @@ SELECT cast(1 as decimal(5, 0)) = cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7788,7 +7788,7 @@ SELECT cast(1 as decimal(10, 0)) = cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7812,7 +7812,7 @@ SELECT cast(1 as decimal(20, 0)) = cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7868,7 +7868,7 @@ SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7892,7 +7892,7 @@ SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7916,7 +7916,7 @@ SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7940,7 +7940,7 @@ SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7964,7 +7964,7 @@ SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -7988,7 +7988,7 @@ SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8012,7 +8012,7 @@ SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8036,7 +8036,7 @@ SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8284,7 +8284,7 @@ SELECT cast('1' as binary) <=> cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8308,7 +8308,7 @@ SELECT cast('1' as binary) <=> cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8332,7 +8332,7 @@ SELECT cast('1' as binary) <=> cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8356,7 +8356,7 @@ SELECT cast('1' as binary) <=> cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8380,7 +8380,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(3, 0)) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8404,7 +8404,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(5, 0)) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8428,7 +8428,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(10, 0)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8452,7 +8452,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(20, 0)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8476,7 +8476,7 @@ SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8500,7 +8500,7 @@ SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8524,7 +8524,7 @@ SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8548,7 +8548,7 @@ SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8828,7 +8828,7 @@ SELECT cast(1 as decimal(3, 0)) <=> cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8852,7 +8852,7 @@ SELECT cast(1 as decimal(5, 0)) <=> cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8876,7 +8876,7 @@ SELECT cast(1 as decimal(10, 0)) <=> cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8900,7 +8900,7 @@ SELECT cast(1 as decimal(20, 0)) <=> cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8956,7 +8956,7 @@ SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -8980,7 +8980,7 @@ SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9004,7 +9004,7 @@ SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9028,7 +9028,7 @@ SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9052,7 +9052,7 @@ SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9076,7 +9076,7 @@ SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9100,7 +9100,7 @@ SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9124,7 +9124,7 @@ SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9372,7 +9372,7 @@ SELECT cast('1' as binary) < cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9396,7 +9396,7 @@ SELECT cast('1' as binary) < cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9420,7 +9420,7 @@ SELECT cast('1' as binary) < cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9444,7 +9444,7 @@ SELECT cast('1' as binary) < cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9468,7 +9468,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(3, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9492,7 +9492,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(5, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9516,7 +9516,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(10, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9540,7 +9540,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(20, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9564,7 +9564,7 @@ SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9588,7 +9588,7 @@ SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9612,7 +9612,7 @@ SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9636,7 +9636,7 @@ SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9916,7 +9916,7 @@ SELECT cast(1 as decimal(3, 0)) < cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9940,7 +9940,7 @@ SELECT cast(1 as decimal(5, 0)) < cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9964,7 +9964,7 @@ SELECT cast(1 as decimal(10, 0)) < cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -9988,7 +9988,7 @@ SELECT cast(1 as decimal(20, 0)) < cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10012,7 +10012,7 @@ SELECT cast(1 as decimal(3, 0)) < cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10036,7 +10036,7 @@ SELECT cast(1 as decimal(5, 0)) < cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10060,7 +10060,7 @@ SELECT cast(1 as decimal(10, 0)) < cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10084,7 +10084,7 @@ SELECT cast(1 as decimal(20, 0)) < cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10108,7 +10108,7 @@ SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10132,7 +10132,7 @@ SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10156,7 +10156,7 @@ SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10180,7 +10180,7 @@ SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10204,7 +10204,7 @@ SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10228,7 +10228,7 @@ SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10252,7 +10252,7 @@ SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10276,7 +10276,7 @@ SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10524,7 +10524,7 @@ SELECT cast('1' as binary) <= cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10548,7 +10548,7 @@ SELECT cast('1' as binary) <= cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10572,7 +10572,7 @@ SELECT cast('1' as binary) <= cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10596,7 +10596,7 @@ SELECT cast('1' as binary) <= cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10620,7 +10620,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(3, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10644,7 +10644,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(5, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10668,7 +10668,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(10, 0)) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10692,7 +10692,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(20, 0)) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10716,7 +10716,7 @@ SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10740,7 +10740,7 @@ SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10764,7 +10764,7 @@ SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -10788,7 +10788,7 @@ SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11068,7 +11068,7 @@ SELECT cast(1 as decimal(3, 0)) <= cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11092,7 +11092,7 @@ SELECT cast(1 as decimal(5, 0)) <= cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11116,7 +11116,7 @@ SELECT cast(1 as decimal(10, 0)) <= cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11140,7 +11140,7 @@ SELECT cast(1 as decimal(20, 0)) <= cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11164,7 +11164,7 @@ SELECT cast(1 as decimal(3, 0)) <= cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11188,7 +11188,7 @@ SELECT cast(1 as decimal(5, 0)) <= cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11212,7 +11212,7 @@ SELECT cast(1 as decimal(10, 0)) <= cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11236,7 +11236,7 @@ SELECT cast(1 as decimal(20, 0)) <= cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11260,7 +11260,7 @@ SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11284,7 +11284,7 @@ SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11308,7 +11308,7 @@ SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11332,7 +11332,7 @@ SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11356,7 +11356,7 @@ SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11380,7 +11380,7 @@ SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11404,7 +11404,7 @@ SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11428,7 +11428,7 @@ SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11676,7 +11676,7 @@ SELECT cast('1' as binary) > cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11700,7 +11700,7 @@ SELECT cast('1' as binary) > cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11724,7 +11724,7 @@ SELECT cast('1' as binary) > cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11748,7 +11748,7 @@ SELECT cast('1' as binary) > cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11772,7 +11772,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(3, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11796,7 +11796,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(5, 0)) FRO -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11820,7 +11820,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(10, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11844,7 +11844,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(20, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11868,7 +11868,7 @@ SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11892,7 +11892,7 @@ SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11916,7 +11916,7 @@ SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -11940,7 +11940,7 @@ SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12220,7 +12220,7 @@ SELECT cast(1 as decimal(3, 0)) > cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12244,7 +12244,7 @@ SELECT cast(1 as decimal(5, 0)) > cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12268,7 +12268,7 @@ SELECT cast(1 as decimal(10, 0)) > cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12292,7 +12292,7 @@ SELECT cast(1 as decimal(20, 0)) > cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12316,7 +12316,7 @@ SELECT cast(1 as decimal(3, 0)) > cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12340,7 +12340,7 @@ SELECT cast(1 as decimal(5, 0)) > cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12364,7 +12364,7 @@ SELECT cast(1 as decimal(10, 0)) > cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12388,7 +12388,7 @@ SELECT cast(1 as decimal(20, 0)) > cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12412,7 +12412,7 @@ SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12436,7 +12436,7 @@ SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12460,7 +12460,7 @@ SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12484,7 +12484,7 @@ SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12508,7 +12508,7 @@ SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12532,7 +12532,7 @@ SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12556,7 +12556,7 @@ SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12580,7 +12580,7 @@ SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12828,7 +12828,7 @@ SELECT cast('1' as binary) >= cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12852,7 +12852,7 @@ SELECT cast('1' as binary) >= cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12876,7 +12876,7 @@ SELECT cast('1' as binary) >= cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12900,7 +12900,7 @@ SELECT cast('1' as binary) >= cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12924,7 +12924,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(3, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12948,7 +12948,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(5, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12972,7 +12972,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(10, 0)) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -12996,7 +12996,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(20, 0)) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13020,7 +13020,7 @@ SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13044,7 +13044,7 @@ SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13068,7 +13068,7 @@ SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13092,7 +13092,7 @@ SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13372,7 +13372,7 @@ SELECT cast(1 as decimal(3, 0)) >= cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13396,7 +13396,7 @@ SELECT cast(1 as decimal(5, 0)) >= cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13420,7 +13420,7 @@ SELECT cast(1 as decimal(10, 0)) >= cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13444,7 +13444,7 @@ SELECT cast(1 as decimal(20, 0)) >= cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13468,7 +13468,7 @@ SELECT cast(1 as decimal(3, 0)) >= cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13492,7 +13492,7 @@ SELECT cast(1 as decimal(5, 0)) >= cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13516,7 +13516,7 @@ SELECT cast(1 as decimal(10, 0)) >= cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13540,7 +13540,7 @@ SELECT cast(1 as decimal(20, 0)) >= cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13564,7 +13564,7 @@ SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13588,7 +13588,7 @@ SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13612,7 +13612,7 @@ SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13636,7 +13636,7 @@ SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13660,7 +13660,7 @@ SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13684,7 +13684,7 @@ SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13708,7 +13708,7 @@ SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13732,7 +13732,7 @@ SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -13980,7 +13980,7 @@ SELECT cast('1' as binary) <> cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14004,7 +14004,7 @@ SELECT cast('1' as binary) <> cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14028,7 +14028,7 @@ SELECT cast('1' as binary) <> cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14052,7 +14052,7 @@ SELECT cast('1' as binary) <> cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14076,7 +14076,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(3, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14100,7 +14100,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(5, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14124,7 +14124,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(10, 0)) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14148,7 +14148,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(20, 0)) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14172,7 +14172,7 @@ SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(3, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14196,7 +14196,7 @@ SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(5, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14220,7 +14220,7 @@ SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14244,7 +14244,7 @@ SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(20, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14524,7 +14524,7 @@ SELECT cast(1 as decimal(3, 0)) <> cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14548,7 +14548,7 @@ SELECT cast(1 as decimal(5, 0)) <> cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14572,7 +14572,7 @@ SELECT cast(1 as decimal(10, 0)) <> cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14596,7 +14596,7 @@ SELECT cast(1 as decimal(20, 0)) <> cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14652,7 +14652,7 @@ SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14676,7 +14676,7 @@ SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14700,7 +14700,7 @@ SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14724,7 +14724,7 @@ SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14748,7 +14748,7 @@ SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14772,7 +14772,7 @@ SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14796,7 +14796,7 @@ SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -14820,7 +14820,7 @@ SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", 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 00a738a9a9a0..5f16135fcaf4 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 @@ -76,7 +76,7 @@ SELECT cast(1 as tinyint) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -100,7 +100,7 @@ SELECT cast(1 as tinyint) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -124,7 +124,7 @@ SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -148,7 +148,7 @@ SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -236,7 +236,7 @@ SELECT cast(1 as smallint) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -260,7 +260,7 @@ SELECT cast(1 as smallint) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -284,7 +284,7 @@ SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -308,7 +308,7 @@ SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -396,7 +396,7 @@ SELECT cast(1 as int) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -420,7 +420,7 @@ SELECT cast(1 as int) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -444,7 +444,7 @@ SELECT cast(1 as int) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -468,7 +468,7 @@ SELECT cast(1 as int) / cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -556,7 +556,7 @@ SELECT cast(1 as bigint) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -580,7 +580,7 @@ SELECT cast(1 as bigint) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -604,7 +604,7 @@ SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -628,7 +628,7 @@ SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -716,7 +716,7 @@ SELECT cast(1 as float) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -740,7 +740,7 @@ SELECT cast(1 as float) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -764,7 +764,7 @@ SELECT cast(1 as float) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -788,7 +788,7 @@ SELECT cast(1 as float) / cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -876,7 +876,7 @@ SELECT cast(1 as double) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -900,7 +900,7 @@ SELECT cast(1 as double) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -924,7 +924,7 @@ SELECT cast(1 as double) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -948,7 +948,7 @@ SELECT cast(1 as double) / cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1036,7 +1036,7 @@ SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1060,7 +1060,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1084,7 +1084,7 @@ SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00.0' as timestamp) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1108,7 +1108,7 @@ SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1196,7 +1196,7 @@ SELECT cast(1 as string) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1220,7 +1220,7 @@ SELECT cast(1 as string) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1244,7 +1244,7 @@ SELECT cast(1 as string) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1268,7 +1268,7 @@ SELECT cast(1 as string) / cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1292,7 +1292,7 @@ SELECT cast('1' as binary) / cast(1 as tinyint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1316,7 +1316,7 @@ SELECT cast('1' as binary) / cast(1 as smallint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1340,7 +1340,7 @@ SELECT cast('1' as binary) / cast(1 as int) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1364,7 +1364,7 @@ SELECT cast('1' as binary) / cast(1 as bigint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1388,7 +1388,7 @@ SELECT cast('1' as binary) / cast(1 as float) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1412,7 +1412,7 @@ SELECT cast('1' as binary) / cast(1 as double) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1436,7 +1436,7 @@ SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1460,7 +1460,7 @@ SELECT cast('1' as binary) / cast(1 as string) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1484,7 +1484,7 @@ SELECT cast('1' as binary) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", @@ -1508,7 +1508,7 @@ SELECT cast('1' as binary) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1532,7 +1532,7 @@ SELECT cast('1' as binary) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1556,7 +1556,7 @@ SELECT cast('1' as binary) / cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1580,7 +1580,7 @@ SELECT cast(1 as boolean) / cast(1 as tinyint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1604,7 +1604,7 @@ SELECT cast(1 as boolean) / cast(1 as smallint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1628,7 +1628,7 @@ SELECT cast(1 as boolean) / cast(1 as int) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1652,7 +1652,7 @@ SELECT cast(1 as boolean) / cast(1 as bigint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1676,7 +1676,7 @@ SELECT cast(1 as boolean) / cast(1 as float) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1700,7 +1700,7 @@ SELECT cast(1 as boolean) / cast(1 as double) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1724,7 +1724,7 @@ SELECT cast(1 as boolean) / cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1748,7 +1748,7 @@ SELECT cast(1 as boolean) / cast(1 as string) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1772,7 +1772,7 @@ SELECT cast(1 as boolean) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1796,7 +1796,7 @@ SELECT cast(1 as boolean) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", @@ -1820,7 +1820,7 @@ SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1844,7 +1844,7 @@ SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1868,7 +1868,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as tinyint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1892,7 +1892,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as smallint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1916,7 +1916,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as int) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1940,7 +1940,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as bigint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1964,7 +1964,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as float) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1988,7 +1988,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as double) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2012,7 +2012,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2036,7 +2036,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as string) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2060,7 +2060,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2084,7 +2084,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2108,7 +2108,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00.0' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", @@ -2132,7 +2132,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00' a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2156,7 +2156,7 @@ SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as tinyint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2180,7 +2180,7 @@ SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as smallint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2204,7 +2204,7 @@ SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as int) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2228,7 +2228,7 @@ SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as bigint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2252,7 +2252,7 @@ SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as float) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2276,7 +2276,7 @@ SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as double) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2300,7 +2300,7 @@ SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2324,7 +2324,7 @@ SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as string) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2348,7 +2348,7 @@ SELECT cast('2017-12-11 09:30:00' as date) / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2372,7 +2372,7 @@ SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2396,7 +2396,7 @@ SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00.0' as tim -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -2420,7 +2420,7 @@ SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00' as date) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out index c40b49a16f55..c0c3cefab841 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out @@ -76,7 +76,7 @@ SELECT IF(true, cast(1 as tinyint), cast('2' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -100,7 +100,7 @@ SELECT IF(true, cast(1 as tinyint), cast(2 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -124,7 +124,7 @@ SELECT IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as timestamp)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -148,7 +148,7 @@ SELECT IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -236,7 +236,7 @@ SELECT IF(true, cast(1 as smallint), cast('2' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -260,7 +260,7 @@ SELECT IF(true, cast(1 as smallint), cast(2 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -284,7 +284,7 @@ SELECT IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00.0' as timestamp)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -308,7 +308,7 @@ SELECT IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -396,7 +396,7 @@ SELECT IF(true, cast(1 as int), cast('2' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -420,7 +420,7 @@ SELECT IF(true, cast(1 as int), cast(2 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -444,7 +444,7 @@ SELECT IF(true, cast(1 as int), cast('2017-12-11 09:30:00.0' as timestamp)) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -468,7 +468,7 @@ SELECT IF(true, cast(1 as int), cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -556,7 +556,7 @@ SELECT IF(true, cast(1 as bigint), cast('2' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -580,7 +580,7 @@ SELECT IF(true, cast(1 as bigint), cast(2 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -604,7 +604,7 @@ SELECT IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00.0' as timestamp)) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -628,7 +628,7 @@ SELECT IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -716,7 +716,7 @@ SELECT IF(true, cast(1 as float), cast('2' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -740,7 +740,7 @@ SELECT IF(true, cast(1 as float), cast(2 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -764,7 +764,7 @@ SELECT IF(true, cast(1 as float), cast('2017-12-11 09:30:00.0' as timestamp)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -788,7 +788,7 @@ SELECT IF(true, cast(1 as float), cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -876,7 +876,7 @@ SELECT IF(true, cast(1 as double), cast('2' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -900,7 +900,7 @@ SELECT IF(true, cast(1 as double), cast(2 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -924,7 +924,7 @@ SELECT IF(true, cast(1 as double), cast('2017-12-11 09:30:00.0' as timestamp)) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -948,7 +948,7 @@ SELECT IF(true, cast(1 as double), cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1036,7 +1036,7 @@ SELECT IF(true, cast(1 as decimal(10, 0)), cast('2' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1060,7 +1060,7 @@ SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1084,7 +1084,7 @@ SELECT IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as times -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1108,7 +1108,7 @@ SELECT IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1196,7 +1196,7 @@ SELECT IF(true, cast(1 as string), cast('2' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1220,7 +1220,7 @@ SELECT IF(true, cast(1 as string), cast(2 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1260,7 +1260,7 @@ SELECT IF(true, cast('1' as binary), cast(2 as tinyint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1284,7 +1284,7 @@ SELECT IF(true, cast('1' as binary), cast(2 as smallint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1308,7 +1308,7 @@ SELECT IF(true, cast('1' as binary), cast(2 as int)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1332,7 +1332,7 @@ SELECT IF(true, cast('1' as binary), cast(2 as bigint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1356,7 +1356,7 @@ SELECT IF(true, cast('1' as binary), cast(2 as float)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1380,7 +1380,7 @@ SELECT IF(true, cast('1' as binary), cast(2 as double)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1404,7 +1404,7 @@ SELECT IF(true, cast('1' as binary), cast(2 as decimal(10, 0))) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1428,7 +1428,7 @@ SELECT IF(true, cast('1' as binary), cast(2 as string)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1460,7 +1460,7 @@ SELECT IF(true, cast('1' as binary), cast(2 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1484,7 +1484,7 @@ SELECT IF(true, cast('1' as binary), cast('2017-12-11 09:30:00.0' as timestamp)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1508,7 +1508,7 @@ SELECT IF(true, cast('1' as binary), cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1532,7 +1532,7 @@ SELECT IF(true, cast(1 as boolean), cast(2 as tinyint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1556,7 +1556,7 @@ SELECT IF(true, cast(1 as boolean), cast(2 as smallint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1580,7 +1580,7 @@ SELECT IF(true, cast(1 as boolean), cast(2 as int)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1604,7 +1604,7 @@ SELECT IF(true, cast(1 as boolean), cast(2 as bigint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1628,7 +1628,7 @@ SELECT IF(true, cast(1 as boolean), cast(2 as float)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1652,7 +1652,7 @@ SELECT IF(true, cast(1 as boolean), cast(2 as double)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1676,7 +1676,7 @@ SELECT IF(true, cast(1 as boolean), cast(2 as decimal(10, 0))) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1700,7 +1700,7 @@ SELECT IF(true, cast(1 as boolean), cast(2 as string)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1724,7 +1724,7 @@ SELECT IF(true, cast(1 as boolean), cast('2' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1756,7 +1756,7 @@ SELECT IF(true, cast(1 as boolean), cast('2017-12-11 09:30:00.0' as timestamp)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1780,7 +1780,7 @@ SELECT IF(true, cast(1 as boolean), cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1804,7 +1804,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as tinyint)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1828,7 +1828,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as smallint)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1852,7 +1852,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as int)) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1876,7 +1876,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as bigint)) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1900,7 +1900,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as float)) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1924,7 +1924,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as double)) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1948,7 +1948,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as decimal(10 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1980,7 +1980,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast('2' as binary)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2004,7 +2004,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as boolean)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2044,7 +2044,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as tinyint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2068,7 +2068,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as smallint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2092,7 +2092,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as int)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2116,7 +2116,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as bigint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2140,7 +2140,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as float)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2164,7 +2164,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as double)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2188,7 +2188,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as decimal(10, 0))) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2220,7 +2220,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast('2' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2244,7 +2244,7 @@ SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out index 65738778f2be..3ca78fa17a64 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out @@ -76,7 +76,7 @@ SELECT cast(1 as tinyint) in (cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -100,7 +100,7 @@ SELECT cast(1 as tinyint) in (cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -124,7 +124,7 @@ SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -148,7 +148,7 @@ SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -236,7 +236,7 @@ SELECT cast(1 as smallint) in (cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -260,7 +260,7 @@ SELECT cast(1 as smallint) in (cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -284,7 +284,7 @@ SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -308,7 +308,7 @@ SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -396,7 +396,7 @@ SELECT cast(1 as int) in (cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -420,7 +420,7 @@ SELECT cast(1 as int) in (cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -444,7 +444,7 @@ SELECT cast(1 as int) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -468,7 +468,7 @@ SELECT cast(1 as int) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -556,7 +556,7 @@ SELECT cast(1 as bigint) in (cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -580,7 +580,7 @@ SELECT cast(1 as bigint) in (cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -604,7 +604,7 @@ SELECT cast(1 as bigint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -628,7 +628,7 @@ SELECT cast(1 as bigint) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -716,7 +716,7 @@ SELECT cast(1 as float) in (cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -740,7 +740,7 @@ SELECT cast(1 as float) in (cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -764,7 +764,7 @@ SELECT cast(1 as float) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -788,7 +788,7 @@ SELECT cast(1 as float) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -876,7 +876,7 @@ SELECT cast(1 as double) in (cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -900,7 +900,7 @@ SELECT cast(1 as double) in (cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -924,7 +924,7 @@ SELECT cast(1 as double) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -948,7 +948,7 @@ SELECT cast(1 as double) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1036,7 +1036,7 @@ SELECT cast(1 as decimal(10, 0)) in (cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1060,7 +1060,7 @@ SELECT cast(1 as decimal(10, 0)) in (cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1084,7 +1084,7 @@ SELECT cast(1 as decimal(10, 0)) in (cast('2017-12-11 09:30:00.0' as timestamp)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1108,7 +1108,7 @@ SELECT cast(1 as decimal(10, 0)) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1196,7 +1196,7 @@ SELECT cast(1 as string) in (cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1220,7 +1220,7 @@ SELECT cast(1 as string) in (cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1260,7 +1260,7 @@ SELECT cast('1' as binary) in (cast(1 as tinyint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1284,7 +1284,7 @@ SELECT cast('1' as binary) in (cast(1 as smallint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1308,7 +1308,7 @@ SELECT cast('1' as binary) in (cast(1 as int)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1332,7 +1332,7 @@ SELECT cast('1' as binary) in (cast(1 as bigint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1356,7 +1356,7 @@ SELECT cast('1' as binary) in (cast(1 as float)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1380,7 +1380,7 @@ SELECT cast('1' as binary) in (cast(1 as double)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1404,7 +1404,7 @@ SELECT cast('1' as binary) in (cast(1 as decimal(10, 0))) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1428,7 +1428,7 @@ SELECT cast('1' as binary) in (cast(1 as string)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1460,7 +1460,7 @@ SELECT cast('1' as binary) in (cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1484,7 +1484,7 @@ SELECT cast('1' as binary) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1508,7 +1508,7 @@ SELECT cast('1' as binary) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1532,7 +1532,7 @@ SELECT true in (cast(1 as tinyint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1556,7 +1556,7 @@ SELECT true in (cast(1 as smallint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1580,7 +1580,7 @@ SELECT true in (cast(1 as int)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1604,7 +1604,7 @@ SELECT true in (cast(1 as bigint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1628,7 +1628,7 @@ SELECT true in (cast(1 as float)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1652,7 +1652,7 @@ SELECT true in (cast(1 as double)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1676,7 +1676,7 @@ SELECT true in (cast(1 as decimal(10, 0))) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1700,7 +1700,7 @@ SELECT true in (cast(1 as string)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1724,7 +1724,7 @@ SELECT true in (cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1756,7 +1756,7 @@ SELECT true in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1780,7 +1780,7 @@ SELECT true in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1804,7 +1804,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as tinyint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1828,7 +1828,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as smallint)) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1852,7 +1852,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as int)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1876,7 +1876,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as bigint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1900,7 +1900,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as float)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1924,7 +1924,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as double)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1948,7 +1948,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as decimal(10, 0))) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -1980,7 +1980,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2' as binary)) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2004,7 +2004,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2044,7 +2044,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as tinyint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2068,7 +2068,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as smallint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2092,7 +2092,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as int)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2116,7 +2116,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as bigint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2140,7 +2140,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as float)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2164,7 +2164,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as double)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2188,7 +2188,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as decimal(10, 0))) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2220,7 +2220,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast('2' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2244,7 +2244,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2348,7 +2348,7 @@ SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2372,7 +2372,7 @@ SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2396,7 +2396,7 @@ SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('2017-12-11 09:30:00.0' a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2420,7 +2420,7 @@ SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('2017-12-11 09:30:00' as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2508,7 +2508,7 @@ SELECT cast(1 as smallint) in (cast(1 as smallint), cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2532,7 +2532,7 @@ SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2556,7 +2556,7 @@ SELECT cast(1 as smallint) in (cast(1 as smallint), cast('2017-12-11 09:30:00.0' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2580,7 +2580,7 @@ SELECT cast(1 as smallint) in (cast(1 as smallint), cast('2017-12-11 09:30:00' a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2668,7 +2668,7 @@ SELECT cast(1 as int) in (cast(1 as int), cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2692,7 +2692,7 @@ SELECT cast(1 as int) in (cast(1 as int), cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2716,7 +2716,7 @@ SELECT cast(1 as int) in (cast(1 as int), cast('2017-12-11 09:30:00.0' as timest -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2740,7 +2740,7 @@ SELECT cast(1 as int) in (cast(1 as int), cast('2017-12-11 09:30:00' as date)) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2828,7 +2828,7 @@ SELECT cast(1 as bigint) in (cast(1 as bigint), cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2852,7 +2852,7 @@ SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2876,7 +2876,7 @@ SELECT cast(1 as bigint) in (cast(1 as bigint), cast('2017-12-11 09:30:00.0' as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2900,7 +2900,7 @@ SELECT cast(1 as bigint) in (cast(1 as bigint), cast('2017-12-11 09:30:00' as da -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -2988,7 +2988,7 @@ SELECT cast(1 as float) in (cast(1 as float), cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3012,7 +3012,7 @@ SELECT cast(1 as float) in (cast(1 as float), cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3036,7 +3036,7 @@ SELECT cast(1 as float) in (cast(1 as float), cast('2017-12-11 09:30:00.0' as ti -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3060,7 +3060,7 @@ SELECT cast(1 as float) in (cast(1 as float), cast('2017-12-11 09:30:00' as date -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3148,7 +3148,7 @@ SELECT cast(1 as double) in (cast(1 as double), cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3172,7 +3172,7 @@ SELECT cast(1 as double) in (cast(1 as double), cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3196,7 +3196,7 @@ SELECT cast(1 as double) in (cast(1 as double), cast('2017-12-11 09:30:00.0' as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3220,7 +3220,7 @@ SELECT cast(1 as double) in (cast(1 as double), cast('2017-12-11 09:30:00' as da -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3308,7 +3308,7 @@ SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('1' as bina -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3332,7 +3332,7 @@ SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as boolea -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3356,7 +3356,7 @@ SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('2017-12-11 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3380,7 +3380,7 @@ SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('2017-12-11 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3468,7 +3468,7 @@ SELECT cast(1 as string) in (cast(1 as string), cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3492,7 +3492,7 @@ SELECT cast(1 as string) in (cast(1 as string), cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3532,7 +3532,7 @@ SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as tinyint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3556,7 +3556,7 @@ SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as smallint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3580,7 +3580,7 @@ SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as int)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3604,7 +3604,7 @@ SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as bigint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3628,7 +3628,7 @@ SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as float)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3652,7 +3652,7 @@ SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as double)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3676,7 +3676,7 @@ SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as decimal(10, 0))) F -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3700,7 +3700,7 @@ SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as string)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3732,7 +3732,7 @@ SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3756,7 +3756,7 @@ SELECT cast('1' as binary) in (cast('1' as binary), cast('2017-12-11 09:30:00.0' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3780,7 +3780,7 @@ SELECT cast('1' as binary) in (cast('1' as binary), cast('2017-12-11 09:30:00' a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3804,7 +3804,7 @@ SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as tinyint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3828,7 +3828,7 @@ SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as smallint)) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3852,7 +3852,7 @@ SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as int)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3876,7 +3876,7 @@ SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as bigint)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3900,7 +3900,7 @@ SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as float)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3924,7 +3924,7 @@ SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as double)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3948,7 +3948,7 @@ SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as decimal(10, 0))) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3972,7 +3972,7 @@ SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as string)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -3996,7 +3996,7 @@ SELECT cast('1' as boolean) in (cast('1' as boolean), cast('1' as binary)) FROM -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4028,7 +4028,7 @@ SELECT cast('1' as boolean) in (cast('1' as boolean), cast('2017-12-11 09:30:00. -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4052,7 +4052,7 @@ SELECT cast('1' as boolean) in (cast('1' as boolean), cast('2017-12-11 09:30:00' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4076,7 +4076,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00. -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4100,7 +4100,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00. -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4124,7 +4124,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00. -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4148,7 +4148,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00. -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4172,7 +4172,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00. -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4196,7 +4196,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00. -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4220,7 +4220,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00. -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4252,7 +4252,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00. -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4276,7 +4276,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00. -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4316,7 +4316,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as dat -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4340,7 +4340,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as dat -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4364,7 +4364,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as dat -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4388,7 +4388,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as dat -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4412,7 +4412,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as dat -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4436,7 +4436,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as dat -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4460,7 +4460,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as dat -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4492,7 +4492,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as dat -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -4516,7 +4516,7 @@ SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as dat -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out index f476b701bfee..452603c2b1eb 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out @@ -81,7 +81,7 @@ FROM various_maps -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", "sqlState" : "42K09", @@ -125,7 +125,7 @@ FROM various_maps -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out index d9ef3ad4b587..5907f700877f 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out @@ -90,7 +90,7 @@ FROM various_maps -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -116,7 +116,7 @@ FROM various_maps -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -142,7 +142,7 @@ FROM various_maps -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -168,7 +168,7 @@ FROM various_maps -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", @@ -194,7 +194,7 @@ FROM various_maps -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", "sqlState" : "42K09", 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 19ad8c0e1130..8b87d9228bca 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 @@ -76,7 +76,7 @@ SELECT '1' + cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -100,7 +100,7 @@ SELECT '1' + cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -124,7 +124,7 @@ SELECT '1' + cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -148,7 +148,7 @@ SELECT '1' + cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -238,7 +238,7 @@ SELECT '1' - cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -262,7 +262,7 @@ SELECT '1' - cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -286,7 +286,7 @@ SELECT '1' - cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -384,7 +384,7 @@ SELECT '1' * cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -408,7 +408,7 @@ SELECT '1' * cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -432,7 +432,7 @@ SELECT '1' * cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -456,7 +456,7 @@ SELECT '1' * cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -544,7 +544,7 @@ SELECT '1' / cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -568,7 +568,7 @@ SELECT '1' / cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -592,7 +592,7 @@ SELECT '1' / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -616,7 +616,7 @@ SELECT '1' / cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -704,7 +704,7 @@ SELECT '1' % cast('1' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -728,7 +728,7 @@ SELECT '1' % cast(1 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -752,7 +752,7 @@ SELECT '1' % cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -776,7 +776,7 @@ SELECT '1' % cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -864,7 +864,7 @@ SELECT pmod('1', cast('1' as binary)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -888,7 +888,7 @@ SELECT pmod('1', cast(1 as boolean)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -912,7 +912,7 @@ SELECT pmod('1', cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -936,7 +936,7 @@ SELECT pmod('1', cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1016,7 +1016,7 @@ SELECT cast('1' as binary) + '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1040,7 +1040,7 @@ SELECT cast(1 as boolean) + '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1064,7 +1064,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) + '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1088,7 +1088,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1170,7 +1170,7 @@ SELECT cast('1' as binary) - '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1194,7 +1194,7 @@ SELECT cast(1 as boolean) - '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1218,7 +1218,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1244,7 +1244,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -1326,7 +1326,7 @@ SELECT cast('1' as binary) * '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1350,7 +1350,7 @@ SELECT cast(1 as boolean) * '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1374,7 +1374,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) * '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1398,7 +1398,7 @@ SELECT cast('2017-12-11 09:30:00' as date) * '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1478,7 +1478,7 @@ SELECT cast('1' as binary) / '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1502,7 +1502,7 @@ SELECT cast(1 as boolean) / '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1526,7 +1526,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1550,7 +1550,7 @@ SELECT cast('2017-12-11 09:30:00' as date) / '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1630,7 +1630,7 @@ SELECT cast('1' as binary) % '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1654,7 +1654,7 @@ SELECT cast(1 as boolean) % '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1678,7 +1678,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) % '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1702,7 +1702,7 @@ SELECT cast('2017-12-11 09:30:00' as date) % '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1782,7 +1782,7 @@ SELECT pmod(cast('1' as binary), '1') FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1806,7 +1806,7 @@ SELECT pmod(cast(1 as boolean), '1') FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1830,7 +1830,7 @@ SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), '1') FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", @@ -1854,7 +1854,7 @@ SELECT pmod(cast('2017-12-11 09:30:00' as date), '1') FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out index 3a367f6b3f3c..08b70ff920eb 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out @@ -100,7 +100,7 @@ select cast(a as array) from t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", "sqlState" : "42K09", @@ -124,7 +124,7 @@ select cast(a as struct) from t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", "sqlState" : "42K09", @@ -148,7 +148,7 @@ select cast(a as map) from t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", "sqlState" : "42K09", diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out index e474a494d483..79184489758b 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out @@ -84,7 +84,7 @@ SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -111,7 +111,7 @@ SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -138,7 +138,7 @@ SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as ti -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -165,7 +165,7 @@ SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -264,7 +264,7 @@ SELECT cast(1 as smallint) FROM t UNION SELECT cast('2' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -291,7 +291,7 @@ SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -318,7 +318,7 @@ SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -345,7 +345,7 @@ SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as dat -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -444,7 +444,7 @@ SELECT cast(1 as int) FROM t UNION SELECT cast('2' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -471,7 +471,7 @@ SELECT cast(1 as int) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -498,7 +498,7 @@ SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timest -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -525,7 +525,7 @@ SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -624,7 +624,7 @@ SELECT cast(1 as bigint) FROM t UNION SELECT cast('2' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -651,7 +651,7 @@ SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -678,7 +678,7 @@ SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as tim -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -705,7 +705,7 @@ SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -804,7 +804,7 @@ SELECT cast(1 as float) FROM t UNION SELECT cast('2' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -831,7 +831,7 @@ SELECT cast(1 as float) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -858,7 +858,7 @@ SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as time -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -885,7 +885,7 @@ SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -984,7 +984,7 @@ SELECT cast(1 as double) FROM t UNION SELECT cast('2' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1011,7 +1011,7 @@ SELECT cast(1 as double) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1038,7 +1038,7 @@ SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as tim -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1065,7 +1065,7 @@ SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1164,7 +1164,7 @@ SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1191,7 +1191,7 @@ SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1218,7 +1218,7 @@ SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00.0 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1245,7 +1245,7 @@ SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00' -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1344,7 +1344,7 @@ SELECT cast(1 as string) FROM t UNION SELECT cast('2' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1371,7 +1371,7 @@ SELECT cast(1 as string) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1416,7 +1416,7 @@ SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as tinyint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1443,7 +1443,7 @@ SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as smallint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1470,7 +1470,7 @@ SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as int) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1497,7 +1497,7 @@ SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as bigint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1524,7 +1524,7 @@ SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as float) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1551,7 +1551,7 @@ SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as double) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1578,7 +1578,7 @@ SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1605,7 +1605,7 @@ SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as string) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1641,7 +1641,7 @@ SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1668,7 +1668,7 @@ SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1695,7 +1695,7 @@ SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00' as dat -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1722,7 +1722,7 @@ SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as tinyint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1749,7 +1749,7 @@ SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as smallint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1776,7 +1776,7 @@ SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as int) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1803,7 +1803,7 @@ SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as bigint) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1830,7 +1830,7 @@ SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as float) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1857,7 +1857,7 @@ SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as double) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1884,7 +1884,7 @@ SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1911,7 +1911,7 @@ SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as string) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1938,7 +1938,7 @@ SELECT cast(1 as boolean) FROM t UNION SELECT cast('2' as binary) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -1973,7 +1973,7 @@ SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as ti -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2000,7 +2000,7 @@ SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2027,7 +2027,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2054,7 +2054,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2081,7 +2081,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2108,7 +2108,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2135,7 +2135,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2162,7 +2162,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2189,7 +2189,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2225,7 +2225,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2' a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2252,7 +2252,7 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2297,7 +2297,7 @@ SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as tinyint -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2324,7 +2324,7 @@ SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as smallin -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2351,7 +2351,7 @@ SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as int) FR -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2378,7 +2378,7 @@ SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as bigint) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2405,7 +2405,7 @@ SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as float) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2432,7 +2432,7 @@ SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as double) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2459,7 +2459,7 @@ SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as decimal -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2495,7 +2495,7 @@ SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2' as binar -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -2522,7 +2522,7 @@ SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as boolean -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out index 43e6873f40d8..f6d1120c75ff 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out @@ -164,7 +164,7 @@ SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string) DESC RANGE BETWE -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", "sqlState" : "42K09", @@ -189,7 +189,7 @@ SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary) DESC RANGE BET -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", "sqlState" : "42K09", @@ -214,7 +214,7 @@ SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean) DESC RANGE BETW -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", "sqlState" : "42K09", @@ -239,7 +239,7 @@ SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as ti -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", "sqlState" : "42K09", 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 8a9a22791712..6e955c1e4f1e 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 @@ -498,7 +498,7 @@ from tenk1 o -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out index a97d734e548b..e31538c16642 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out @@ -4,7 +4,7 @@ select udf(max(min(unique1))) from tenk1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NESTED_AGGREGATE_FUNCTION", "sqlState" : "42607", 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 a757ae83d244..8a74362c167f 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 @@ -1956,7 +1956,15 @@ CREATE TABLE t1 (name STRING, n INTEGER) USING parquet -- !query schema struct<> -- !query output - +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "LOCATION_ALREADY_EXISTS", + "sqlState" : "42710", + "messageParameters" : { + "identifier" : "`spark_catalog`.`default`.`t1`", + "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" + } +} -- !query @@ -1980,7 +1988,21 @@ INSERT INTO t1 VALUES ( 'bb', 11 ) -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 14, + "fragment" : "t1" + } ] +} -- !query @@ -2034,12 +2056,23 @@ struct<> -- !query SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name) -- !query schema -struct +struct<> -- !query output -bb 11 12 13 -cc NULL 22 23 -dd NULL NULL 33 -ee NULL 42 NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 16, + "fragment" : "t1" + } ] +} -- !query @@ -2131,9 +2164,23 @@ NATURAL INNER JOIN NATURAL INNER JOIN (SELECT udf(udf(udf(name))) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3 -- !query schema -struct +struct<> -- !query output -bb 11 1 12 2 13 3 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 78, + "stopIndex" : 79, + "fragment" : "t1" + } ] +} -- !query @@ -2144,12 +2191,23 @@ NATURAL FULL JOIN NATURAL FULL JOIN (SELECT udf(udf(name)) as name, udf(n) as s3_n, udf(3) as s3_2 FROM t3) s3 -- !query schema -struct +struct<> -- !query output -bb 11 1 12 2 13 3 -cc NULL NULL 22 2 23 3 -dd NULL NULL NULL NULL 33 3 -ee NULL NULL 42 2 NULL NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 83, + "stopIndex" : 84, + "fragment" : "t1" + } ] +} -- !query @@ -2162,12 +2220,23 @@ NATURAL FULL JOIN (SELECT name, udf(udf(n)) as s3_n FROM t3) as s3 ) ss2 -- !query schema -struct +struct<> -- !query output -bb 11 12 13 -cc NULL 22 23 -dd NULL NULL 33 -ee NULL 42 NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 95, + "stopIndex" : 96, + "fragment" : "t1" + } ] +} -- !query @@ -2180,12 +2249,23 @@ NATURAL FULL JOIN (SELECT name, udf(n) as s3_n FROM t3) as s3 ) ss2 -- !query schema -struct +struct<> -- !query output -bb 11 12 2 13 -cc NULL 22 2 23 -dd NULL NULL NULL 33 -ee NULL 42 2 NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 45, + "fragment" : "t1" + } ] +} -- !query @@ -2195,12 +2275,23 @@ FULL JOIN (SELECT name, 2 as s2_n FROM t2) as s2 ON (udf(udf(s1_n)) = udf(s2_n)) -- !query schema -struct +struct<> -- !query output -NULL NULL bb 2 -NULL NULL cc 2 -NULL NULL ee 2 -bb 11 NULL NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 93, + "stopIndex" : 94, + "fragment" : "t1" + } ] +} -- !query @@ -2475,7 +2566,14 @@ DROP TABLE t1 -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`t1`" + } +} -- !query @@ -3300,7 +3398,7 @@ select * from -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3333,7 +3431,7 @@ select udf(t1.uunique1) from -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3357,7 +3455,7 @@ select udf(udf(t2.uunique1)) from -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3381,7 +3479,7 @@ select udf(uunique1) from -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -3595,7 +3693,7 @@ select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -3617,7 +3715,7 @@ select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -3639,7 +3737,7 @@ select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", @@ -3661,7 +3759,7 @@ select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", "sqlState" : "42703", 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 1f07f5d1c610..a220e6487f81 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 @@ -139,7 +139,7 @@ SELECT udf(a) FROM test_having HAVING udf(min(a)) < udf(max(a)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -158,7 +158,7 @@ SELECT 1 AS one FROM test_having HAVING udf(a) > 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", 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 0ee95e786f52..a3a7cee4eaa7 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 @@ -121,7 +121,7 @@ SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -368,7 +368,7 @@ SELECT udf(count(udf(a))) FROM test_missing_target GROUP BY udf(a) ORDER BY udf( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out index 24e77ca3e07b..ad1b0aabcdec 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out @@ -137,7 +137,7 @@ SELECT array(1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -227,7 +227,7 @@ SELECT k, v FROM tab4 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NUM_COLUMNS_MISMATCH", "sqlState" : "42826", diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out index f927bf036633..ffa25eecdce4 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out @@ -206,7 +206,7 @@ SELECT course, udf(year), GROUPING(course) FROM courseSales GROUP BY course, udf -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", "queryContext" : [ { @@ -224,7 +224,7 @@ SELECT course, udf(year), GROUPING_ID(course, year) FROM courseSales GROUP BY ud -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", "queryContext" : [ { 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 9b5979440045..e1737de2b670 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 @@ -14,7 +14,7 @@ SELECT udf(a), udf(COUNT(b)) FROM testData -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -52,7 +52,7 @@ SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -123,7 +123,7 @@ SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -187,7 +187,7 @@ SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "GROUP_BY_AGGREGATE", "sqlState" : "42903", @@ -218,7 +218,7 @@ SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -242,7 +242,7 @@ SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -328,7 +328,7 @@ SELECT udf(id) FROM range(10) HAVING id > 0 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", @@ -444,7 +444,7 @@ SELECT every(udf(1)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -470,7 +470,7 @@ SELECT some(udf(1S)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -496,7 +496,7 @@ SELECT any(udf(1L)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -522,7 +522,7 @@ SELECT udf(every("true")) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", @@ -625,7 +625,7 @@ SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WHERE_CONDITION", "sqlState" : "42903", @@ -648,7 +648,7 @@ SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WHERE_CONDITION", "sqlState" : "42903", @@ -671,7 +671,7 @@ SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_WHERE_CONDITION", "sqlState" : "42903", diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out index 05d089744dac..240469b0cab6 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out @@ -94,7 +94,7 @@ SELECT array(1), udf(2) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", "sqlState" : "42825", @@ -123,7 +123,7 @@ SELECT udf(k), udf(v) FROM tab2 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NUM_COLUMNS_MISMATCH", "sqlState" : "42826", 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 8bd2157b453a..50376fb86a50 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 @@ -238,7 +238,7 @@ PIVOT ( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -282,7 +282,7 @@ PIVOT ( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "NESTED_AGGREGATE_FUNCTION", "sqlState" : "42607", @@ -364,7 +364,7 @@ PIVOT ( -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out index 3fa08c2e4358..451f514a2170 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out @@ -57,7 +57,7 @@ ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY udf(ca -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", "sqlState" : "42K09", @@ -203,7 +203,7 @@ ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, u -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_INVALID_BOUND", "sqlState" : "42K09", @@ -228,7 +228,7 @@ RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", "sqlState" : "42K09", @@ -251,7 +251,7 @@ RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_MULTI_ORDER", "sqlState" : "42K09", @@ -275,7 +275,7 @@ RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", "sqlState" : "42K09", @@ -300,7 +300,7 @@ RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY udf(cate), val -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_WRONG_COMPARISON", "sqlState" : "42K09", From 751fc872dd0fb2fe008e4efe8f86e96aa8891550 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 25 Jul 2023 16:24:49 -0400 Subject: [PATCH 09/13] Fix verbatims --- .../analyzer-results/charvarchar.sql.out | 157 +---- .../sql-tests/analyzer-results/hll.sql.out | 30 +- .../analyzer-results/null-handling.sql.out | 518 +++------------- .../analyzer-results/postgreSQL/join.sql.out | 221 +++---- .../postgreSQL/window_part3.sql.out | 116 +--- .../udf/postgreSQL/udf-join.sql.out | 221 +++---- .../sql-tests/results/charvarchar.sql.out | 261 ++++---- .../resources/sql-tests/results/hll.sql.out | 28 +- .../sql-tests/results/null-handling.sql.out | 576 ++++-------------- .../sql-tests/results/postgreSQL/join.sql.out | 158 +---- .../results/postgreSQL/window_part3.sql.out | 104 +--- .../results/udf/postgreSQL/udf-join.sql.out | 158 +---- 12 files changed, 686 insertions(+), 1862 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out index c773231d84d9..6e72fd28686a 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out @@ -239,13 +239,12 @@ DescribeTableCommand `spark_catalog`.`default`.`char_tbl1`, true, [col_name#x, d -- !query create table char_part(c1 char(5), c2 char(2), v1 varchar(6), v2 varchar(2)) using parquet partitioned by (v2, c2) -- !query analysis -org.apache.spark.SparkRuntimeException +org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException { - "errorClass" : "LOCATION_ALREADY_EXISTS", - "sqlState" : "42710", + "errorClass" : "TABLE_OR_VIEW_ALREADY_EXISTS", + "sqlState" : "42P07", "messageParameters" : { - "identifier" : "`spark_catalog`.`default`.`char_part`", - "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/char_part'" + "relationName" : "`spark_catalog`.`default`.`char_part`" } } @@ -253,181 +252,71 @@ org.apache.spark.SparkRuntimeException -- !query desc formatted char_part -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 24, - "fragment" : "char_part" - } ] -} +DescribeTableCommand `spark_catalog`.`default`.`char_part`, true, [col_name#x, data_type#x, comment#x] -- !query alter table char_part add partition (v2='ke', c2='nt') location 'loc1' -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.catalyst.analysis.PartitionsAlreadyExistException { - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", + "errorClass" : "PARTITIONS_ALREADY_EXIST", + "sqlState" : "428FT", "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 21, - "fragment" : "char_part" - } ] + "partitionList" : "PARTITION (`v2` = ke, `c2` = nt)", + "tableName" : "`default`.`char_part`" + } } -- !query desc formatted char_part -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 24, - "fragment" : "char_part" - } ] -} +DescribeTableCommand `spark_catalog`.`default`.`char_part`, true, [col_name#x, data_type#x, comment#x] -- !query alter table char_part partition (v2='ke') rename to partition (v2='nt') -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", + "errorClass" : "_LEGACY_ERROR_TEMP_1232", "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 21, - "fragment" : "char_part" - } ] + "partitionColumnNames" : "v2, c2", + "specKeys" : "v2", + "tableName" : "`spark_catalog`.`default`.`char_part`" + } } -- !query desc formatted char_part -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 24, - "fragment" : "char_part" - } ] -} +DescribeTableCommand `spark_catalog`.`default`.`char_part`, true, [col_name#x, data_type#x, comment#x] -- !query alter table char_part partition (v2='ke', c2='nt') set location 'loc2' -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 21, - "fragment" : "char_part" - } ] -} +AlterTableSetLocationCommand `spark_catalog`.`default`.`char_part`, Map(v2 -> ke, c2 -> nt), loc2 -- !query desc formatted char_part -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 24, - "fragment" : "char_part" - } ] -} +DescribeTableCommand `spark_catalog`.`default`.`char_part`, true, [col_name#x, data_type#x, comment#x] -- !query MSCK REPAIR TABLE char_part -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 27, - "fragment" : "char_part" - } ] -} +RepairTableCommand `spark_catalog`.`default`.`char_part`, true, false, MSCK REPAIR TABLE -- !query desc formatted char_part -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 24, - "fragment" : "char_part" - } ] -} +DescribeTableCommand `spark_catalog`.`default`.`char_part`, true, [col_name#x, data_type#x, comment#x] -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/hll.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/hll.sql.out index 1360458202f4..cd6fcc7c459b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/hll.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/hll.sql.out @@ -9,35 +9,17 @@ DropTable true, false -- !query CREATE TABLE t1 USING JSON AS VALUES (0), (1), (2), (2), (2), (3), (4) as tab(col) -- !query analysis -org.apache.spark.SparkRuntimeException -{ - "errorClass" : "LOCATION_ALREADY_EXISTS", - "sqlState" : "42710", - "messageParameters" : { - "identifier" : "`spark_catalog`.`default`.`t1`", - "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" - } -} +CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`t1`, ErrorIfExists, [col] + +- SubqueryAlias tab + +- LocalRelation [col#x] -- !query SELECT hll_sketch_estimate(hll_sketch_agg(col)) AS result FROM t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 64, - "stopIndex" : 65, - "fragment" : "t1" - } ] -} +Aggregate [hll_sketch_estimate(hll_sketch_agg(col#x, 12, 0, 0)) AS result#xL] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[col#x] json -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out index 9a1e0c52f33e..26e9394932a1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out @@ -2,565 +2,239 @@ -- !query create table t1(a int, b int, c int) using parquet -- !query analysis -org.apache.spark.SparkRuntimeException -{ - "errorClass" : "LOCATION_ALREADY_EXISTS", - "sqlState" : "42710", - "messageParameters" : { - "identifier" : "`spark_catalog`.`default`.`t1`", - "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" - } -} +CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false -- !query insert into t1 values(1,0,0) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] -- !query insert into t1 values(2,0,1) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] -- !query insert into t1 values(3,1,0) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] -- !query insert into t1 values(4,1,1) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] -- !query insert into t1 values(5,null,0) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] -- !query insert into t1 values(6,null,1) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] -- !query insert into t1 values(7,null,null) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c] ++- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x] + +- LocalRelation [col1#x, col2#x, col3#x] -- !query select a, b+c from t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 20, - "stopIndex" : 21, - "fragment" : "t1" - } ] -} +Project [a#x, (b#x + c#x) AS (b + c)#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select a+10, b*0 from t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 23, - "stopIndex" : 24, - "fragment" : "t1" - } ] -} +Project [(a#x + 10) AS (a + 10)#x, (b#x * 0) AS (b * 0)#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select distinct b from t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 24, - "stopIndex" : 25, - "fragment" : "t1" - } ] -} +Distinct ++- Project [b#x] + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select b from t1 union select b from t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 16, - "fragment" : "t1" - } ] -} +Distinct ++- Union false, false + :- Project [b#x] + : +- SubqueryAlias spark_catalog.default.t1 + : +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + +- Project [b#x] + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select a+20, case b when c then 1 else 0 end from t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 51, - "stopIndex" : 52, - "fragment" : "t1" - } ] -} +Project [(a#x + 20) AS (a + 20)#x, CASE WHEN (b#x = c#x) THEN 1 ELSE 0 END AS CASE WHEN (b = c) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select a+30, case c when b then 1 else 0 end from t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 51, - "stopIndex" : 52, - "fragment" : "t1" - } ] -} +Project [(a#x + 30) AS (a + 30)#x, CASE WHEN (c#x = b#x) THEN 1 ELSE 0 END AS CASE WHEN (c = b) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select a+40, case when b<>0 then 1 else 0 end from t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 52, - "stopIndex" : 53, - "fragment" : "t1" - } ] -} +Project [(a#x + 40) AS (a + 40)#x, CASE WHEN NOT (b#x = 0) THEN 1 ELSE 0 END AS CASE WHEN (NOT (b = 0)) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select a+50, case when not b<>0 then 1 else 0 end from t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 56, - "stopIndex" : 57, - "fragment" : "t1" - } ] -} +Project [(a#x + 50) AS (a + 50)#x, CASE WHEN NOT NOT (b#x = 0) THEN 1 ELSE 0 END AS CASE WHEN (NOT (NOT (b = 0))) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select a+60, case when b<>0 and c<>0 then 1 else 0 end from t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 61, - "stopIndex" : 62, - "fragment" : "t1" - } ] -} +Project [(a#x + 60) AS (a + 60)#x, CASE WHEN (NOT (b#x = 0) AND NOT (c#x = 0)) THEN 1 ELSE 0 END AS CASE WHEN ((NOT (b = 0)) AND (NOT (c = 0))) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select a+70, case when not (b<>0 and c<>0) then 1 else 0 end from t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 67, - "stopIndex" : 68, - "fragment" : "t1" - } ] -} +Project [(a#x + 70) AS (a + 70)#x, CASE WHEN NOT (NOT (b#x = 0) AND NOT (c#x = 0)) THEN 1 ELSE 0 END AS CASE WHEN (NOT ((NOT (b = 0)) AND (NOT (c = 0)))) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select a+80, case when b<>0 or c<>0 then 1 else 0 end from t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 60, - "stopIndex" : 61, - "fragment" : "t1" - } ] -} +Project [(a#x + 80) AS (a + 80)#x, CASE WHEN (NOT (b#x = 0) OR NOT (c#x = 0)) THEN 1 ELSE 0 END AS CASE WHEN ((NOT (b = 0)) OR (NOT (c = 0))) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select a+90, case when not (b<>0 or c<>0) then 1 else 0 end from t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 66, - "stopIndex" : 67, - "fragment" : "t1" - } ] -} +Project [(a#x + 90) AS (a + 90)#x, CASE WHEN NOT (NOT (b#x = 0) OR NOT (c#x = 0)) THEN 1 ELSE 0 END AS CASE WHEN (NOT ((NOT (b = 0)) OR (NOT (c = 0)))) THEN 1 ELSE 0 END#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select count(*), count(b), sum(b), avg(b), min(b), max(b) from t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 64, - "stopIndex" : 65, - "fragment" : "t1" - } ] -} +Aggregate [count(1) AS count(1)#xL, count(b#x) AS count(b)#xL, sum(b#x) AS sum(b)#xL, avg(b#x) AS avg(b)#x, min(b#x) AS min(b)#x, max(b#x) AS max(b)#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select a+100 from t1 where b<10 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 20, - "fragment" : "t1" - } ] -} +Project [(a#x + 100) AS (a + 100)#x] ++- Filter (b#x < 10) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select a+110 from t1 where not b>10 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 20, - "fragment" : "t1" - } ] -} +Project [(a#x + 110) AS (a + 110)#x] ++- Filter NOT (b#x > 10) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select a+120 from t1 where b<10 OR c=1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 20, - "fragment" : "t1" - } ] -} +Project [(a#x + 120) AS (a + 120)#x] ++- Filter ((b#x < 10) OR (c#x = 1)) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select a+130 from t1 where b<10 AND c=1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 20, - "fragment" : "t1" - } ] -} +Project [(a#x + 130) AS (a + 130)#x] ++- Filter ((b#x < 10) AND (c#x = 1)) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select a+140 from t1 where not (b<10 AND c=1) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 20, - "fragment" : "t1" - } ] -} +Project [(a#x + 140) AS (a + 140)#x] ++- Filter NOT ((b#x < 10) AND (c#x = 1)) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select a+150 from t1 where not (c=1 AND b<10) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 20, - "fragment" : "t1" - } ] -} +Project [(a#x + 150) AS (a + 150)#x] ++- Filter NOT ((c#x = 1) AND (b#x < 10)) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query select b, c, equal_null(b, c), equal_null(c, b) from t1 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 54, - "stopIndex" : 55, - "fragment" : "t1" - } ] -} +Project [b#x, c#x, equal_null(b#x, c#x) AS equal_null(b, c)#x, equal_null(c#x, b#x) AS equal_null(c, b)#x] ++- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet -- !query drop table t1 -- !query analysis -org.apache.spark.sql.catalyst.analysis.NoSuchTableException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`spark_catalog`.`default`.`t1`" - } -} +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out index 7809cbb57009..cf5c79333e83 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out @@ -655,15 +655,7 @@ Project [ AS xxx#x, i#x, j#x, t#x, k#x] -- !query CREATE TABLE t1 (name STRING, n INTEGER) USING parquet -- !query analysis -org.apache.spark.SparkRuntimeException -{ - "errorClass" : "LOCATION_ALREADY_EXISTS", - "sqlState" : "42710", - "messageParameters" : { - "identifier" : "`spark_catalog`.`default`.`t1`", - "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" - } -} +CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false -- !query @@ -681,21 +673,9 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t3`, false -- !query INSERT INTO t1 VALUES ( 'bb', 11 ) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] -- !query @@ -749,21 +729,17 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d -- !query SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 16, - "fragment" : "t1" - } ] -} +Project [name#x, n#x, n#x, n#x] ++- Project [coalesce(name#x, name#x) AS name#x, n#x, n#x, n#x] + +- Join FullOuter, (name#x = name#x) + :- Project [coalesce(name#x, name#x) AS name#x, n#x, n#x] + : +- Join FullOuter, (name#x = name#x) + : :- SubqueryAlias spark_catalog.default.t1 + : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet -- !query @@ -891,21 +867,23 @@ NATURAL INNER JOIN NATURAL INNER JOIN (SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 55, - "stopIndex" : 56, - "fragment" : "t1" - } ] -} +Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] ++- Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] + +- Join Inner, (name#x = name#x) + :- Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x] + : +- Join Inner, (name#x = name#x) + : :- SubqueryAlias s1 + : : +- Project [name#x, n#x AS s1_n#x, 1 AS s1_1#x] + : : +- SubqueryAlias spark_catalog.default.t1 + : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + : +- SubqueryAlias s2 + : +- Project [name#x, n#x AS s2_n#x, 2 AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, n#x AS s3_n#x, 3 AS s3_2#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet -- !query @@ -916,21 +894,23 @@ NATURAL FULL JOIN NATURAL FULL JOIN (SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 55, - "stopIndex" : 56, - "fragment" : "t1" - } ] -} +Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] ++- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] + +- Join FullOuter, (name#x = name#x) + :- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x] + : +- Join FullOuter, (name#x = name#x) + : :- SubqueryAlias s1 + : : +- Project [name#x, n#x AS s1_n#x, 1 AS s1_1#x] + : : +- SubqueryAlias spark_catalog.default.t1 + : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + : +- SubqueryAlias s2 + : +- Project [name#x, n#x AS s2_n#x, 2 AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, n#x AS s3_n#x, 3 AS s3_2#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet -- !query @@ -943,21 +923,25 @@ NATURAL FULL JOIN (SELECT name, n as s3_n FROM t3) as s3 ) ss2 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 44, - "stopIndex" : 45, - "fragment" : "t1" - } ] -} +Project [name#x, s1_n#x, s2_n#x, s3_n#x] ++- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s2_n#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s1 + : +- Project [name#x, n#x AS s1_n#x] + : +- SubqueryAlias spark_catalog.default.t1 + : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + +- SubqueryAlias ss2 + +- Project [name#x, s2_n#x, s3_n#x] + +- Project [coalesce(name#x, name#x) AS name#x, s2_n#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [name#x, n#x AS s2_n#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, n#x AS s3_n#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet -- !query @@ -970,21 +954,25 @@ NATURAL FULL JOIN (SELECT name, n as s3_n FROM t3) as s3 ) ss2 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 44, - "stopIndex" : 45, - "fragment" : "t1" - } ] -} +Project [name#x, s1_n#x, s2_n#x, s2_2#x, s3_n#x] ++- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s2_n#x, s2_2#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s1 + : +- Project [name#x, n#x AS s1_n#x] + : +- SubqueryAlias spark_catalog.default.t1 + : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + +- SubqueryAlias ss2 + +- Project [name#x, s2_n#x, s2_2#x, s3_n#x] + +- Project [coalesce(name#x, name#x) AS name#x, s2_n#x, s2_2#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [name#x, n#x AS s2_n#x, 2 AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, n#x AS s3_n#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet -- !query @@ -994,21 +982,16 @@ FULL JOIN (SELECT name, 2 as s2_n FROM t2) as s2 ON (s1_n = s2_n) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 46, - "stopIndex" : 47, - "fragment" : "t1" - } ] -} +Project [name#x, s1_n#x, name#x, s2_n#x] ++- Join FullOuter, (s1_n#x = s2_n#x) + :- SubqueryAlias s1 + : +- Project [name#x, n#x AS s1_n#x] + : +- SubqueryAlias spark_catalog.default.t1 + : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + +- SubqueryAlias s2 + +- Project [name#x, 2 AS s2_n#x] + +- SubqueryAlias spark_catalog.default.t2 + +- Relation spark_catalog.default.t2[name#x,n#x] parquet -- !query @@ -1603,14 +1586,8 @@ Aggregate [count(1) AS count(1)#xL] -- !query DROP TABLE t1 -- !query analysis -org.apache.spark.sql.catalyst.analysis.NoSuchTableException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`spark_catalog`.`default`.`t1`" - } -} +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1 -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out index f77be5a62b67..6b6a37b4e7fb 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out @@ -215,35 +215,15 @@ GlobalLimit 0 -- !query create table t1 (f1 int, f2 int) using parquet -- !query analysis -org.apache.spark.SparkRuntimeException -{ - "errorClass" : "LOCATION_ALREADY_EXISTS", - "sqlState" : "42710", - "messageParameters" : { - "identifier" : "`spark_catalog`.`default`.`t1`", - "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" - } -} +CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false -- !query insert into t1 values (1,1),(1,2),(2,2) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [f1, f2] ++- Project [cast(col1#x as int) AS f1#x, cast(col2#x as int) AS f2#x] + +- LocalRelation [col1#x, col2#x] -- !query @@ -253,17 +233,17 @@ from t1 where f1 = f2 -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", + "sqlState" : "42K09", "messageParameters" : { - "relationName" : "`t1`" + "sqlExpr" : "\"(PARTITION BY f1 RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 115, - "stopIndex" : 116, - "fragment" : "t1" + "startIndex" : 25, + "stopIndex" : 108, + "fragment" : "(partition by f1\n range between 1 preceding and 1 following)" } ] } @@ -273,21 +253,13 @@ select f1, sum(f1) over (partition by f1 order by f2 range between 1 preceding and 1 following) from t1 where f1 = f2 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 102, - "stopIndex" : 103, - "fragment" : "t1" - } ] -} +Project [f1#x, sum(f1) OVER (PARTITION BY f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL] ++- Project [f1#x, f2#x, sum(f1) OVER (PARTITION BY f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL, sum(f1) OVER (PARTITION BY f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL] + +- Window [sum(f1#x) windowspecdefinition(f1#x, f2#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -1, 1)) AS sum(f1) OVER (PARTITION BY f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [f1#x], [f2#x ASC NULLS FIRST] + +- Project [f1#x, f2#x] + +- Filter (f1#x = f2#x) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[f1#x,f2#x] parquet -- !query @@ -295,21 +267,13 @@ select f1, sum(f1) over (partition by f1, f1 order by f2 range between 2 preceding and 1 preceding) from t1 where f1 = f2 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 106, - "stopIndex" : 107, - "fragment" : "t1" - } ] -} +Project [f1#x, sum(f1) OVER (PARTITION BY f1, f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL] ++- Project [f1#x, f2#x, sum(f1) OVER (PARTITION BY f1, f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL, sum(f1) OVER (PARTITION BY f1, f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL] + +- Window [sum(f1#x) windowspecdefinition(f1#x, f1#x, f2#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -2, -1)) AS sum(f1) OVER (PARTITION BY f1, f1 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 2 PRECEDING AND 1 PRECEDING)#xL], [f1#x, f1#x], [f2#x ASC NULLS FIRST] + +- Project [f1#x, f2#x] + +- Filter (f1#x = f2#x) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[f1#x,f2#x] parquet -- !query @@ -317,21 +281,13 @@ select f1, sum(f1) over (partition by f1, f2 order by f2 range between 1 following and 2 following) from t1 where f1 = f2 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 106, - "stopIndex" : 107, - "fragment" : "t1" - } ] -} +Project [f1#x, sum(f1) OVER (PARTITION BY f1, f2 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 FOLLOWING AND 2 FOLLOWING)#xL] ++- Project [f1#x, f2#x, sum(f1) OVER (PARTITION BY f1, f2 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 FOLLOWING AND 2 FOLLOWING)#xL, sum(f1) OVER (PARTITION BY f1, f2 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 FOLLOWING AND 2 FOLLOWING)#xL] + +- Window [sum(f1#x) windowspecdefinition(f1#x, f2#x, f2#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, 1, 2)) AS sum(f1) OVER (PARTITION BY f1, f2 ORDER BY f2 ASC NULLS FIRST RANGE BETWEEN 1 FOLLOWING AND 2 FOLLOWING)#xL], [f1#x, f2#x], [f2#x ASC NULLS FIRST] + +- Project [f1#x, f2#x] + +- Filter (f1#x = f2#x) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[f1#x,f2#x] parquet -- !query @@ -530,11 +486,5 @@ DropTable false, false -- !query DROP TABLE t1 -- !query analysis -org.apache.spark.sql.catalyst.analysis.NoSuchTableException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`spark_catalog`.`default`.`t1`" - } -} +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1 diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out index 02d3facbfaa2..e5f51cd80bf0 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out @@ -659,15 +659,7 @@ Project [cast(udf(cast( as string)) as string) AS xxx#x, cast(udf(cast(i#x as st -- !query CREATE TABLE t1 (name STRING, n INTEGER) USING parquet -- !query analysis -org.apache.spark.SparkRuntimeException -{ - "errorClass" : "LOCATION_ALREADY_EXISTS", - "sqlState" : "42710", - "messageParameters" : { - "identifier" : "`spark_catalog`.`default`.`t1`", - "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" - } -} +CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false -- !query @@ -685,21 +677,9 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t3`, false -- !query INSERT INTO t1 VALUES ( 'bb', 11 ) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [name, n] ++- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x] + +- LocalRelation [col1#x, col2#x] -- !query @@ -753,21 +733,17 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d -- !query SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 16, - "fragment" : "t1" - } ] -} +Project [name#x, n#x, n#x, n#x] ++- Project [coalesce(name#x, name#x) AS name#x, n#x, n#x, n#x] + +- Join FullOuter, (name#x = name#x) + :- Project [coalesce(name#x, name#x) AS name#x, n#x, n#x] + : +- Join FullOuter, (name#x = name#x) + : :- SubqueryAlias spark_catalog.default.t1 + : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet -- !query @@ -895,21 +871,23 @@ NATURAL INNER JOIN NATURAL INNER JOIN (SELECT udf(udf(udf(name))) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 78, - "stopIndex" : 79, - "fragment" : "t1" - } ] -} +Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] ++- Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] + +- Join Inner, (name#x = name#x) + :- Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x] + : +- Join Inner, (name#x = name#x) + : :- SubqueryAlias s1 + : : +- Project [cast(udf(cast(cast(udf(cast(name#x as string)) as string) as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s1_n#x, 1 AS s1_1#x] + : : +- SubqueryAlias spark_catalog.default.t1 + : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + : +- SubqueryAlias s2 + : +- Project [cast(udf(cast(name#x as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s2_n#x, 2 AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [cast(udf(cast(cast(udf(cast(cast(udf(cast(name#x as string)) as string) as string)) as string) as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s3_n#x, 3 AS s3_2#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet -- !query @@ -920,21 +898,23 @@ NATURAL FULL JOIN NATURAL FULL JOIN (SELECT udf(udf(name)) as name, udf(n) as s3_n, udf(3) as s3_2 FROM t3) s3 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 83, - "stopIndex" : 84, - "fragment" : "t1" - } ] -} +Project [name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] ++- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x, s3_n#x, s3_2#x] + +- Join FullOuter, (name#x = name#x) + :- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s1_1#x, s2_n#x, s2_2#x] + : +- Join FullOuter, (name#x = name#x) + : :- SubqueryAlias s1 + : : +- Project [cast(udf(cast(name#x as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s1_n#x, cast(udf(cast(cast(udf(cast(1 as string)) as int) as string)) as int) AS s1_1#x] + : : +- SubqueryAlias spark_catalog.default.t1 + : : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + : +- SubqueryAlias s2 + : +- Project [cast(udf(cast(name#x as string)) as string) AS name#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS s2_n#x, cast(udf(cast(2 as string)) as int) AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [cast(udf(cast(cast(udf(cast(name#x as string)) as string) as string)) as string) AS name#x, cast(udf(cast(n#x as string)) as int) AS s3_n#x, cast(udf(cast(3 as string)) as int) AS s3_2#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet -- !query @@ -947,21 +927,25 @@ NATURAL FULL JOIN (SELECT name, udf(udf(n)) as s3_n FROM t3) as s3 ) ss2 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 95, - "stopIndex" : 96, - "fragment" : "t1" - } ] -} +Project [name#x, cast(udf(cast(cast(udf(cast(s1_n#x as string)) as int) as string)) as int) AS udf(udf(s1_n))#x, cast(udf(cast(s2_n#x as string)) as int) AS udf(s2_n)#x, cast(udf(cast(s3_n#x as string)) as int) AS udf(s3_n)#x] ++- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s2_n#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s1 + : +- Project [name#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS s1_n#x] + : +- SubqueryAlias spark_catalog.default.t1 + : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + +- SubqueryAlias ss2 + +- Project [name#x, s2_n#x, s3_n#x] + +- Project [coalesce(name#x, name#x) AS name#x, s2_n#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [name#x, cast(udf(cast(n#x as string)) as int) AS s2_n#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS s3_n#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet -- !query @@ -974,21 +958,25 @@ NATURAL FULL JOIN (SELECT name, udf(n) as s3_n FROM t3) as s3 ) ss2 -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 44, - "stopIndex" : 45, - "fragment" : "t1" - } ] -} +Project [name#x, s1_n#x, s2_n#x, s2_2#x, s3_n#x] ++- Project [coalesce(name#x, name#x) AS name#x, s1_n#x, s2_n#x, s2_2#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s1 + : +- Project [name#x, n#x AS s1_n#x] + : +- SubqueryAlias spark_catalog.default.t1 + : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + +- SubqueryAlias ss2 + +- Project [name#x, s2_n#x, s2_2#x, s3_n#x] + +- Project [coalesce(name#x, name#x) AS name#x, s2_n#x, s2_2#x, s3_n#x] + +- Join FullOuter, (name#x = name#x) + :- SubqueryAlias s2 + : +- Project [name#x, cast(udf(cast(cast(udf(cast(n#x as string)) as int) as string)) as int) AS s2_n#x, 2 AS s2_2#x] + : +- SubqueryAlias spark_catalog.default.t2 + : +- Relation spark_catalog.default.t2[name#x,n#x] parquet + +- SubqueryAlias s3 + +- Project [name#x, cast(udf(cast(n#x as string)) as int) AS s3_n#x] + +- SubqueryAlias spark_catalog.default.t3 + +- Relation spark_catalog.default.t3[name#x,n#x] parquet -- !query @@ -998,21 +986,16 @@ FULL JOIN (SELECT name, 2 as s2_n FROM t2) as s2 ON (udf(udf(s1_n)) = udf(s2_n)) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 93, - "stopIndex" : 94, - "fragment" : "t1" - } ] -} +Project [name#x, cast(udf(cast(s1_n#x as string)) as int) AS udf(s1_n)#x, name#x, cast(udf(cast(cast(udf(cast(s2_n#x as string)) as int) as string)) as int) AS udf(udf(s2_n))#x] ++- Join FullOuter, (cast(udf(cast(cast(udf(cast(s1_n#x as string)) as int) as string)) as int) = cast(udf(cast(s2_n#x as string)) as int)) + :- SubqueryAlias s1 + : +- Project [name#x, cast(udf(cast(n#x as string)) as int) AS s1_n#x] + : +- SubqueryAlias spark_catalog.default.t1 + : +- Relation spark_catalog.default.t1[name#x,n#x] parquet + +- SubqueryAlias s2 + +- Project [name#x, 2 AS s2_n#x] + +- SubqueryAlias spark_catalog.default.t2 + +- Relation spark_catalog.default.t2[name#x,n#x] parquet -- !query @@ -1607,14 +1590,8 @@ Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL] -- !query DROP TABLE t1 -- !query analysis -org.apache.spark.sql.catalyst.analysis.NoSuchTableException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`spark_catalog`.`default`.`t1`" - } -} +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out index 7681fa95e179..888e8a942891 100644 --- a/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out @@ -513,37 +513,34 @@ create table char_part(c1 char(5), c2 char(2), v1 varchar(6), v2 varchar(2)) usi -- !query schema struct<> -- !query output -org.apache.spark.SparkRuntimeException -{ - "errorClass" : "LOCATION_ALREADY_EXISTS", - "sqlState" : "42710", - "messageParameters" : { - "identifier" : "`spark_catalog`.`default`.`char_part`", - "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/char_part'" - } -} + -- !query desc formatted char_part -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 24, - "fragment" : "char_part" - } ] -} +c1 char(5) +v1 varchar(6) +v2 varchar(2) +c2 char(2) +# Partition Information +# col_name data_type comment +v2 varchar(2) +c2 char(2) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_part +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_part +Partition Provider Catalog -- !query @@ -551,43 +548,34 @@ alter table char_part add partition (v2='ke', c2='nt') location 'loc1' -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 21, - "fragment" : "char_part" - } ] -} + -- !query desc formatted char_part -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 24, - "fragment" : "char_part" - } ] -} +c1 char(5) +v1 varchar(6) +v2 varchar(2) +c2 char(2) +# Partition Information +# col_name data_type comment +v2 varchar(2) +c2 char(2) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_part +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_part +Partition Provider Catalog -- !query @@ -595,43 +583,42 @@ alter table char_part partition (v2='ke') rename to partition (v2='nt') -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", + "errorClass" : "_LEGACY_ERROR_TEMP_1232", "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 21, - "fragment" : "char_part" - } ] + "partitionColumnNames" : "v2, c2", + "specKeys" : "v2", + "tableName" : "`spark_catalog`.`default`.`char_part`" + } } -- !query desc formatted char_part -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 24, - "fragment" : "char_part" - } ] -} +c1 char(5) +v1 varchar(6) +v2 varchar(2) +c2 char(2) +# Partition Information +# col_name data_type comment +v2 varchar(2) +c2 char(2) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_part +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_part +Partition Provider Catalog -- !query @@ -639,43 +626,34 @@ alter table char_part partition (v2='ke', c2='nt') set location 'loc2' -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 21, - "fragment" : "char_part" - } ] -} + -- !query desc formatted char_part -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 24, - "fragment" : "char_part" - } ] -} +c1 char(5) +v1 varchar(6) +v2 varchar(2) +c2 char(2) +# Partition Information +# col_name data_type comment +v2 varchar(2) +c2 char(2) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_part +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_part +Partition Provider Catalog -- !query @@ -683,43 +661,34 @@ MSCK REPAIR TABLE char_part -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 27, - "fragment" : "char_part" - } ] -} + -- !query desc formatted char_part -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`char_part`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 24, - "fragment" : "char_part" - } ] -} +c1 char(5) +v1 varchar(6) +v2 varchar(2) +c2 char(2) +# Partition Information +# col_name data_type comment +v2 varchar(2) +c2 char(2) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_part +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_part +Partition Provider Catalog -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/hll.sql.out b/sql/core/src/test/resources/sql-tests/results/hll.sql.out index c38a57cf8497..8a3f1cb5ea23 100644 --- a/sql/core/src/test/resources/sql-tests/results/hll.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/hll.sql.out @@ -12,37 +12,15 @@ CREATE TABLE t1 USING JSON AS VALUES (0), (1), (2), (2), (2), (3), (4) as tab(co -- !query schema struct<> -- !query output -org.apache.spark.SparkRuntimeException -{ - "errorClass" : "LOCATION_ALREADY_EXISTS", - "sqlState" : "42710", - "messageParameters" : { - "identifier" : "`spark_catalog`.`default`.`t1`", - "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" - } -} + -- !query SELECT hll_sketch_estimate(hll_sketch_agg(col)) AS result FROM t1 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 64, - "stopIndex" : 65, - "fragment" : "t1" - } ] -} +5 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out b/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out index ff72dcd91551..ece6dbef1605 100644 --- a/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out @@ -4,15 +4,7 @@ create table t1(a int, b int, c int) using parquet -- !query schema struct<> -- !query output -org.apache.spark.SparkRuntimeException -{ - "errorClass" : "LOCATION_ALREADY_EXISTS", - "sqlState" : "42710", - "messageParameters" : { - "identifier" : "`spark_catalog`.`default`.`t1`", - "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" - } -} + -- !query @@ -20,21 +12,7 @@ insert into t1 values(1,0,0) -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} + -- !query @@ -42,21 +20,7 @@ insert into t1 values(2,0,1) -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} + -- !query @@ -64,21 +28,7 @@ insert into t1 values(3,1,0) -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} + -- !query @@ -86,21 +36,7 @@ insert into t1 values(4,1,1) -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} + -- !query @@ -108,21 +44,7 @@ insert into t1 values(5,null,0) -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} + -- !query @@ -130,21 +52,7 @@ insert into t1 values(6,null,1) -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} + -- !query @@ -152,461 +60,252 @@ insert into t1 values(7,null,null) -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} + -- !query select a, b+c from t1 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 20, - "stopIndex" : 21, - "fragment" : "t1" - } ] -} +1 0 +2 1 +3 1 +4 2 +5 NULL +6 NULL +7 NULL -- !query select a+10, b*0 from t1 -- !query schema -struct<> +struct<(a + 10):int,(b * 0):int> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 23, - "stopIndex" : 24, - "fragment" : "t1" - } ] -} +11 0 +12 0 +13 0 +14 0 +15 NULL +16 NULL +17 NULL -- !query select distinct b from t1 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 24, - "stopIndex" : 25, - "fragment" : "t1" - } ] -} +0 +1 +NULL -- !query select b from t1 union select b from t1 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 16, - "fragment" : "t1" - } ] -} +0 +1 +NULL -- !query select a+20, case b when c then 1 else 0 end from t1 -- !query schema -struct<> +struct<(a + 20):int,CASE WHEN (b = c) THEN 1 ELSE 0 END:int> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 51, - "stopIndex" : 52, - "fragment" : "t1" - } ] -} +21 1 +22 0 +23 0 +24 1 +25 0 +26 0 +27 0 -- !query select a+30, case c when b then 1 else 0 end from t1 -- !query schema -struct<> +struct<(a + 30):int,CASE WHEN (c = b) THEN 1 ELSE 0 END:int> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 51, - "stopIndex" : 52, - "fragment" : "t1" - } ] -} +31 1 +32 0 +33 0 +34 1 +35 0 +36 0 +37 0 -- !query select a+40, case when b<>0 then 1 else 0 end from t1 -- !query schema -struct<> +struct<(a + 40):int,CASE WHEN (NOT (b = 0)) THEN 1 ELSE 0 END:int> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 52, - "stopIndex" : 53, - "fragment" : "t1" - } ] -} +41 0 +42 0 +43 1 +44 1 +45 0 +46 0 +47 0 -- !query select a+50, case when not b<>0 then 1 else 0 end from t1 -- !query schema -struct<> +struct<(a + 50):int,CASE WHEN (NOT (NOT (b = 0))) THEN 1 ELSE 0 END:int> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 56, - "stopIndex" : 57, - "fragment" : "t1" - } ] -} +51 1 +52 1 +53 0 +54 0 +55 0 +56 0 +57 0 -- !query select a+60, case when b<>0 and c<>0 then 1 else 0 end from t1 -- !query schema -struct<> +struct<(a + 60):int,CASE WHEN ((NOT (b = 0)) AND (NOT (c = 0))) THEN 1 ELSE 0 END:int> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 61, - "stopIndex" : 62, - "fragment" : "t1" - } ] -} +61 0 +62 0 +63 0 +64 1 +65 0 +66 0 +67 0 -- !query select a+70, case when not (b<>0 and c<>0) then 1 else 0 end from t1 -- !query schema -struct<> +struct<(a + 70):int,CASE WHEN (NOT ((NOT (b = 0)) AND (NOT (c = 0)))) THEN 1 ELSE 0 END:int> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 67, - "stopIndex" : 68, - "fragment" : "t1" - } ] -} +71 1 +72 1 +73 1 +74 0 +75 1 +76 0 +77 0 -- !query select a+80, case when b<>0 or c<>0 then 1 else 0 end from t1 -- !query schema -struct<> +struct<(a + 80):int,CASE WHEN ((NOT (b = 0)) OR (NOT (c = 0))) THEN 1 ELSE 0 END:int> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 60, - "stopIndex" : 61, - "fragment" : "t1" - } ] -} +81 0 +82 1 +83 1 +84 1 +85 0 +86 1 +87 0 -- !query select a+90, case when not (b<>0 or c<>0) then 1 else 0 end from t1 -- !query schema -struct<> +struct<(a + 90):int,CASE WHEN (NOT ((NOT (b = 0)) OR (NOT (c = 0)))) THEN 1 ELSE 0 END:int> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 66, - "stopIndex" : 67, - "fragment" : "t1" - } ] -} +91 1 +92 0 +93 0 +94 0 +95 0 +96 0 +97 0 -- !query select count(*), count(b), sum(b), avg(b), min(b), max(b) from t1 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 64, - "stopIndex" : 65, - "fragment" : "t1" - } ] -} +7 4 2 0.5 0 1 -- !query select a+100 from t1 where b<10 -- !query schema -struct<> +struct<(a + 100):int> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 20, - "fragment" : "t1" - } ] -} +101 +102 +103 +104 -- !query select a+110 from t1 where not b>10 -- !query schema -struct<> +struct<(a + 110):int> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 20, - "fragment" : "t1" - } ] -} +111 +112 +113 +114 -- !query select a+120 from t1 where b<10 OR c=1 -- !query schema -struct<> +struct<(a + 120):int> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 20, - "fragment" : "t1" - } ] -} +121 +122 +123 +124 +126 -- !query select a+130 from t1 where b<10 AND c=1 -- !query schema -struct<> +struct<(a + 130):int> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 20, - "fragment" : "t1" - } ] -} +132 +134 -- !query select a+140 from t1 where not (b<10 AND c=1) -- !query schema -struct<> +struct<(a + 140):int> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 20, - "fragment" : "t1" - } ] -} +141 +143 +145 -- !query select a+150 from t1 where not (c=1 AND b<10) -- !query schema -struct<> +struct<(a + 150):int> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 20, - "fragment" : "t1" - } ] -} +151 +153 +155 -- !query select b, c, equal_null(b, c), equal_null(c, b) from t1 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 54, - "stopIndex" : 55, - "fragment" : "t1" - } ] -} +0 0 true true +0 1 false false +1 0 false false +1 1 true true +NULL 0 false false +NULL 1 false false +NULL NULL true true -- !query @@ -614,11 +313,4 @@ drop table t1 -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.analysis.NoSuchTableException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`spark_catalog`.`default`.`t1`" - } -} + 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 0e54db5ecaea..7dd59751b3ae 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 @@ -1956,15 +1956,7 @@ CREATE TABLE t1 (name STRING, n INTEGER) USING parquet -- !query schema struct<> -- !query output -org.apache.spark.SparkRuntimeException -{ - "errorClass" : "LOCATION_ALREADY_EXISTS", - "sqlState" : "42710", - "messageParameters" : { - "identifier" : "`spark_catalog`.`default`.`t1`", - "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" - } -} + -- !query @@ -1988,21 +1980,7 @@ INSERT INTO t1 VALUES ( 'bb', 11 ) -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} + -- !query @@ -2056,23 +2034,12 @@ struct<> -- !query SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 16, - "fragment" : "t1" - } ] -} +bb 11 12 13 +cc NULL 22 23 +dd NULL NULL 33 +ee NULL 42 NULL -- !query @@ -2164,23 +2131,9 @@ NATURAL INNER JOIN NATURAL INNER JOIN (SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 55, - "stopIndex" : 56, - "fragment" : "t1" - } ] -} +bb 11 1 12 2 13 3 -- !query @@ -2191,23 +2144,12 @@ NATURAL FULL JOIN NATURAL FULL JOIN (SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 55, - "stopIndex" : 56, - "fragment" : "t1" - } ] -} +bb 11 1 12 2 13 3 +cc NULL NULL 22 2 23 3 +dd NULL NULL NULL NULL 33 3 +ee NULL NULL 42 2 NULL NULL -- !query @@ -2220,23 +2162,12 @@ NATURAL FULL JOIN (SELECT name, n as s3_n FROM t3) as s3 ) ss2 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 44, - "stopIndex" : 45, - "fragment" : "t1" - } ] -} +bb 11 12 13 +cc NULL 22 23 +dd NULL NULL 33 +ee NULL 42 NULL -- !query @@ -2249,23 +2180,12 @@ NATURAL FULL JOIN (SELECT name, n as s3_n FROM t3) as s3 ) ss2 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 44, - "stopIndex" : 45, - "fragment" : "t1" - } ] -} +bb 11 12 2 13 +cc NULL 22 2 23 +dd NULL NULL NULL 33 +ee NULL 42 2 NULL -- !query @@ -2275,23 +2195,12 @@ FULL JOIN (SELECT name, 2 as s2_n FROM t2) as s2 ON (s1_n = s2_n) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 46, - "stopIndex" : 47, - "fragment" : "t1" - } ] -} +NULL NULL bb 2 +NULL NULL cc 2 +NULL NULL ee 2 +bb 11 NULL NULL -- !query @@ -2566,14 +2475,7 @@ DROP TABLE t1 -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.analysis.NoSuchTableException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`spark_catalog`.`default`.`t1`" - } -} + -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out index 9120a0b0b65b..86ca4e3a0371 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out @@ -235,15 +235,7 @@ create table t1 (f1 int, f2 int) using parquet -- !query schema struct<> -- !query output -org.apache.spark.SparkRuntimeException -{ - "errorClass" : "LOCATION_ALREADY_EXISTS", - "sqlState" : "42710", - "messageParameters" : { - "identifier" : "`spark_catalog`.`default`.`t1`", - "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" - } -} + -- !query @@ -251,21 +243,7 @@ insert into t1 values (1,1),(1,2),(2,2) -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} + -- !query @@ -277,17 +255,17 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", + "sqlState" : "42K09", "messageParameters" : { - "relationName" : "`t1`" + "sqlExpr" : "\"(PARTITION BY f1 RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 115, - "stopIndex" : 116, - "fragment" : "t1" + "startIndex" : 25, + "stopIndex" : 108, + "fragment" : "(partition by f1\n range between 1 preceding and 1 following)" } ] } @@ -297,23 +275,10 @@ select f1, sum(f1) over (partition by f1 order by f2 range between 1 preceding and 1 following) from t1 where f1 = f2 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 102, - "stopIndex" : 103, - "fragment" : "t1" - } ] -} +1 1 +2 2 -- !query @@ -321,23 +286,10 @@ select f1, sum(f1) over (partition by f1, f1 order by f2 range between 2 preceding and 1 preceding) from t1 where f1 = f2 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 106, - "stopIndex" : 107, - "fragment" : "t1" - } ] -} +1 NULL +2 NULL -- !query @@ -345,23 +297,10 @@ select f1, sum(f1) over (partition by f1, f2 order by f2 range between 1 following and 2 following) from t1 where f1 = f2 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 106, - "stopIndex" : 107, - "fragment" : "t1" - } ] -} +1 NULL +2 NULL -- !query @@ -582,11 +521,4 @@ DROP TABLE t1 -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.analysis.NoSuchTableException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`spark_catalog`.`default`.`t1`" - } -} + 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 8a74362c167f..0eb291c07c74 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 @@ -1956,15 +1956,7 @@ CREATE TABLE t1 (name STRING, n INTEGER) USING parquet -- !query schema struct<> -- !query output -org.apache.spark.SparkRuntimeException -{ - "errorClass" : "LOCATION_ALREADY_EXISTS", - "sqlState" : "42710", - "messageParameters" : { - "identifier" : "`spark_catalog`.`default`.`t1`", - "location" : "'file:/Users/herman/Engineering/spark-dev/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/t1'" - } -} + -- !query @@ -1988,21 +1980,7 @@ INSERT INTO t1 VALUES ( 'bb', 11 ) -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 14, - "fragment" : "t1" - } ] -} + -- !query @@ -2056,23 +2034,12 @@ struct<> -- !query SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 16, - "fragment" : "t1" - } ] -} +bb 11 12 13 +cc NULL 22 23 +dd NULL NULL 33 +ee NULL 42 NULL -- !query @@ -2164,23 +2131,9 @@ NATURAL INNER JOIN NATURAL INNER JOIN (SELECT udf(udf(udf(name))) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 78, - "stopIndex" : 79, - "fragment" : "t1" - } ] -} +bb 11 1 12 2 13 3 -- !query @@ -2191,23 +2144,12 @@ NATURAL FULL JOIN NATURAL FULL JOIN (SELECT udf(udf(name)) as name, udf(n) as s3_n, udf(3) as s3_2 FROM t3) s3 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 83, - "stopIndex" : 84, - "fragment" : "t1" - } ] -} +bb 11 1 12 2 13 3 +cc NULL NULL 22 2 23 3 +dd NULL NULL NULL NULL 33 3 +ee NULL NULL 42 2 NULL NULL -- !query @@ -2220,23 +2162,12 @@ NATURAL FULL JOIN (SELECT name, udf(udf(n)) as s3_n FROM t3) as s3 ) ss2 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 95, - "stopIndex" : 96, - "fragment" : "t1" - } ] -} +bb 11 12 13 +cc NULL 22 23 +dd NULL NULL 33 +ee NULL 42 NULL -- !query @@ -2249,23 +2180,12 @@ NATURAL FULL JOIN (SELECT name, udf(n) as s3_n FROM t3) as s3 ) ss2 -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 44, - "stopIndex" : 45, - "fragment" : "t1" - } ] -} +bb 11 12 2 13 +cc NULL 22 2 23 +dd NULL NULL NULL 33 +ee NULL 42 2 NULL -- !query @@ -2275,23 +2195,12 @@ FULL JOIN (SELECT name, 2 as s2_n FROM t2) as s2 ON (udf(udf(s1_n)) = udf(s2_n)) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`t1`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 93, - "stopIndex" : 94, - "fragment" : "t1" - } ] -} +NULL NULL bb 2 +NULL NULL cc 2 +NULL NULL ee 2 +bb 11 NULL NULL -- !query @@ -2566,14 +2475,7 @@ DROP TABLE t1 -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.analysis.NoSuchTableException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`spark_catalog`.`default`.`t1`" - } -} + -- !query From d6df8ab1d2bfde85d4aa346bafcb668c0968d084 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 26 Jul 2023 00:00:45 -0400 Subject: [PATCH 10/13] Fix verbatim --- .../test/resources/sql-tests/results/udaf/udaf-group-by.sql.out | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out index 7725bb529d3d..fe7ceb377e54 100644 --- a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out @@ -14,7 +14,7 @@ SELECT a, udaf(b) FROM testData -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_GROUP_BY", "sqlState" : "42803", From 7ea8c63cfae0d97197184997ba774e6a4112ad9b Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 26 Jul 2023 00:01:52 -0400 Subject: [PATCH 11/13] Fix verbatim --- .../results/udaf/udaf-group-by.sql.out | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out index fe7ceb377e54..f2bb2b8925de 100644 --- a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out @@ -52,7 +52,7 @@ SELECT a, udaf(b) FROM testData GROUP BY b -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -115,7 +115,7 @@ SELECT a + 2, udaf(b) FROM testData GROUP BY a + 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -143,7 +143,7 @@ FROM testData -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_PANDAS_UDF_PLACEMENT", "sqlState" : "0A000", @@ -158,7 +158,7 @@ SELECT udaf(DISTINCT b), udaf(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "_LEGACY_ERROR_TEMP_1023", "messageParameters" : { @@ -200,7 +200,7 @@ SELECT a AS k, udaf(non_existing) FROM testData GROUP BY k -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -232,7 +232,7 @@ SELECT k AS a, udaf(v) FROM testDataHasSameNameWithAlias GROUP BY a -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "MISSING_AGGREGATION", "sqlState" : "42803", @@ -256,7 +256,7 @@ SELECT a AS k, udaf(b) FROM testData GROUP BY k -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", "sqlState" : "42703", @@ -338,7 +338,7 @@ SELECT udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_PANDAS_UDF_PLACEMENT", "sqlState" : "0A000", @@ -353,7 +353,7 @@ SELECT udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_PANDAS_UDF_PLACEMENT", "sqlState" : "0A000", @@ -368,7 +368,7 @@ SELECT udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_PANDAS_UDF_PLACEMENT", "sqlState" : "0A000", @@ -383,7 +383,7 @@ SELECT k, udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "INVALID_PANDAS_UDF_PLACEMENT", "sqlState" : "0A000", From f3ae96a58abb45cfa6293cca99f45a5d4e07d149 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 26 Jul 2023 07:29:02 -0400 Subject: [PATCH 12/13] Fix verbatim --- .../test/resources/sql-tests/results/udaf/udaf-group-by.sql.out | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out index f2bb2b8925de..42155022eb3e 100644 --- a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out @@ -143,7 +143,7 @@ FROM testData -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { "errorClass" : "INVALID_PANDAS_UDF_PLACEMENT", "sqlState" : "0A000", From 1107b5ce4faf14bbd8c78c54d9c97acbc01c4138 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 26 Jul 2023 16:35:40 -0400 Subject: [PATCH 13/13] Verbatims --- .../sql-tests/results/udaf/udaf-group-by.sql.out | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out index 42155022eb3e..84040b6d142c 100644 --- a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out @@ -158,7 +158,7 @@ SELECT udaf(DISTINCT b), udaf(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { "errorClass" : "_LEGACY_ERROR_TEMP_1023", "messageParameters" : { @@ -338,7 +338,7 @@ SELECT udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { "errorClass" : "INVALID_PANDAS_UDF_PLACEMENT", "sqlState" : "0A000", @@ -353,7 +353,7 @@ SELECT udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { "errorClass" : "INVALID_PANDAS_UDF_PLACEMENT", "sqlState" : "0A000", @@ -368,7 +368,7 @@ SELECT udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { "errorClass" : "INVALID_PANDAS_UDF_PLACEMENT", "sqlState" : "0A000", @@ -383,7 +383,7 @@ SELECT k, udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { "errorClass" : "INVALID_PANDAS_UDF_PLACEMENT", "sqlState" : "0A000",